Merge branch 'neo' into 'master'

Change-Id: Id9077fbecad986a67a28d7aa85e1f974fe8a1445
diff --git a/asterixdb/asterix-algebra/pom.xml b/asterixdb/asterix-algebra/pom.xml
index 7a32f42..f542af3 100644
--- a/asterixdb/asterix-algebra/pom.xml
+++ b/asterixdb/asterix-algebra/pom.xml
@@ -202,6 +202,10 @@
       <artifactId>hyracks-api</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-client</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
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 06a6687..fed180b 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
@@ -62,9 +62,12 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeBatchPointSearchCursor;
 
 /**
  * Contributes the runtime operator for an unnest-map representing a BTree search.
@@ -157,7 +160,9 @@
                 nonMatchWriterFactory, dataset, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes,
                 jobGenParams.isLowKeyInclusive(), jobGenParams.isHighKeyInclusive(), propagateFilter,
                 nonFilterWriterFactory, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit,
-                unnestMap.getGenerateCallBackProceedResultVar(), isPrimaryIndexPointSearch(op));
+                unnestMap.getGenerateCallBackProceedResultVar(),
+                isPrimaryIndexPointSearch(op, context.getPhysicalOptimizationConfig()),
+                DefaultTupleProjectorFactory.INSTANCE);
         IOperatorDescriptor opDesc = btreeSearch.first;
         opDesc.setSourceLocation(unnestMap.getSourceLocation());
 
@@ -168,8 +173,12 @@
         builder.contributeGraphEdge(srcExchange, 0, unnestMap, 0);
     }
 
-    private boolean isPrimaryIndexPointSearch(ILogicalOperator op) {
-        if (!isEqCondition || !isPrimaryIndex || !lowKeyVarList.equals(highKeyVarList)) {
+    /**
+     * Check whether we can use {@link LSMBTreeBatchPointSearchCursor} to perform point-lookups on the primary index
+     */
+    private boolean isPrimaryIndexPointSearch(ILogicalOperator op, PhysicalOptimizationConfig config) {
+        if (!config.isBatchLookupEnabled() || !isEqCondition || !isPrimaryIndex
+                || !lowKeyVarList.equals(highKeyVarList)) {
             return false;
         }
         Index searchIndex = ((DataSourceIndex) idx).getIndex();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/DefaultRuleSetFactory.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/DefaultRuleSetFactory.java
index ba42036..67b7910 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/DefaultRuleSetFactory.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/DefaultRuleSetFactory.java
@@ -23,23 +23,33 @@
 
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.optimizer.base.RuleCollections;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialFirstRuleCheckFixpointRuleController;
 import org.apache.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialFixpointRuleController;
 import org.apache.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialOnceRuleController;
 import org.apache.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
 
 public class DefaultRuleSetFactory implements IRuleSetFactory {
 
     @Override
     public List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getLogicalRewrites(
-            ICcApplicationContext appCtx) throws AlgebricksException {
+            ICcApplicationContext appCtx) {
         return buildLogical(appCtx);
     }
 
     @Override
+    public List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getLogicalRewrites(IRuleSetKind ruleSetKind,
+            ICcApplicationContext appCtx) {
+        if (ruleSetKind == RuleSetKind.SAMPLING) {
+            return buildLogicalSampling();
+        } else {
+            throw new IllegalArgumentException(String.valueOf(ruleSetKind));
+        }
+    }
+
+    @Override
     public List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getPhysicalRewrites(
             ICcApplicationContext appCtx) {
         return buildPhysical(appCtx);
@@ -66,9 +76,10 @@
         defaultLogicalRewrites
                 .add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildCondPushDownAndJoinInferenceRuleCollection()));
         defaultLogicalRewrites.add(new Pair<>(seqCtrlFullDfs, RuleCollections.buildLoadFieldsRuleCollection(appCtx)));
-        defaultLogicalRewrites.add(new Pair<>(seqOnceCtrl, RuleCollections.buildDataExchangeRuleCollection()));
-        defaultLogicalRewrites.add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildConsolidationRuleCollection()));
         defaultLogicalRewrites.add(new Pair<>(seqOnceCtrl, RuleCollections.buildFulltextContainsRuleCollection()));
+        defaultLogicalRewrites.add(new Pair<>(seqOnceCtrl, RuleCollections.buildDataExchangeRuleCollection()));
+        defaultLogicalRewrites.add(new Pair<>(seqOnceCtrl, RuleCollections.buildCBORuleCollection()));
+        defaultLogicalRewrites.add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildConsolidationRuleCollection()));
         defaultLogicalRewrites.add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildAccessMethodRuleCollection()));
         defaultLogicalRewrites.add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildPlanCleanupRuleCollection()));
 
@@ -76,6 +87,14 @@
         return defaultLogicalRewrites;
     }
 
+    public static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> buildLogicalSampling() {
+        List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites = new ArrayList<>();
+        SequentialFixpointRuleController seqCtrlNoDfs = new SequentialFixpointRuleController(false);
+        logicalRewrites.add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildConsolidationRuleCollection()));
+        logicalRewrites.add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildPlanCleanupRuleCollection()));
+        return logicalRewrites;
+    }
+
     public static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> buildPhysical(
             ICcApplicationContext appCtx) {
         List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> defaultPhysicalRewrites = new ArrayList<>();
@@ -88,5 +107,4 @@
         defaultPhysicalRewrites.add(new Pair<>(seqOnceCtrl, RuleCollections.prepareForJobGenRuleCollection()));
         return defaultPhysicalRewrites;
     }
-
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java
index f625343..30c6623 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.compiler.provider;
 
+import java.util.Set;
+
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
 import org.apache.asterix.algebra.base.ILangExtension;
 import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
@@ -54,4 +56,9 @@
      * @return the rule set factory of a language implementation
      */
     IRuleSetFactory getRuleSetFactory();
+
+    /**
+     * @return all configurable parameters of a language implementation.
+     */
+    Set<String> getCompilerOptions();
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/IRuleSetFactory.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/IRuleSetFactory.java
index 2300e4a..c643d21 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/IRuleSetFactory.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/IRuleSetFactory.java
@@ -21,24 +21,30 @@
 import java.util.List;
 
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
 
 public interface IRuleSetFactory {
 
+    enum RuleSetKind implements IRuleSetKind {
+        SAMPLING
+    }
+
     /**
      * @return the logical rewrites
-     * @throws AlgebricksException
      */
-    public List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getLogicalRewrites(
-            ICcApplicationContext appCtx) throws AlgebricksException;
+    List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getLogicalRewrites(ICcApplicationContext appCtx);
+
+    /**
+     * @return the logical rewrites of the specified kind
+     */
+    List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getLogicalRewrites(IRuleSetKind ruleSetKind,
+            ICcApplicationContext appCtx);
 
     /**
      * @return the physical rewrites
      */
-    public List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getPhysicalRewrites(
-            ICcApplicationContext appCtx) throws AlgebricksException;
-
+    List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getPhysicalRewrites(ICcApplicationContext appCtx);
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
index 2c18d41..4ad888c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
@@ -18,14 +18,27 @@
  */
 package org.apache.asterix.compiler.provider;
 
+import java.util.HashSet;
+import java.util.Set;
+
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
 import org.apache.asterix.algebra.base.ILangExtension;
+import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.external.feed.watch.FeedActivityDetails;
 import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
 import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.IRewriterFactory;
+import org.apache.asterix.lang.common.statement.StartFeedStatement;
+import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.lang.sqlpp.parser.SqlppParserFactory;
+import org.apache.asterix.lang.sqlpp.rewrites.SqlppQueryRewriter;
 import org.apache.asterix.lang.sqlpp.rewrites.SqlppRewriterFactory;
 import org.apache.asterix.lang.sqlpp.visitor.SqlppAstPrintVisitorFactory;
+import org.apache.asterix.optimizer.base.FuzzyUtils;
+import org.apache.asterix.optimizer.rules.DisjunctivePredicateToJoinRule;
+import org.apache.asterix.optimizer.rules.SetAsterixPhysicalOperatorsRule;
+import org.apache.asterix.optimizer.rules.util.EquivalenceClassUtils;
+import org.apache.asterix.translator.SqlppExpressionToPlanTranslator;
 import org.apache.asterix.translator.SqlppExpressionToPlanTranslatorFactory;
 
 public class SqlppCompilationProvider implements ILangCompilationProvider {
@@ -59,4 +72,26 @@
     public IRuleSetFactory getRuleSetFactory() {
         return new DefaultRuleSetFactory();
     }
+
+    @Override
+    public Set<String> getCompilerOptions() {
+        return new HashSet<>(Set.of(CompilerProperties.COMPILER_JOINMEMORY_KEY,
+                CompilerProperties.COMPILER_GROUPMEMORY_KEY, CompilerProperties.COMPILER_SORTMEMORY_KEY,
+                CompilerProperties.COMPILER_WINDOWMEMORY_KEY, CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
+                CompilerProperties.COMPILER_PARALLELISM_KEY, CompilerProperties.COMPILER_SORT_PARALLEL_KEY,
+                CompilerProperties.COMPILER_SORT_SAMPLES_KEY, CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY,
+                CompilerProperties.COMPILER_INDEXONLY_KEY, CompilerProperties.COMPILER_INTERNAL_SANITYCHECK_KEY,
+                CompilerProperties.COMPILER_EXTERNAL_FIELD_PUSHDOWN_KEY, CompilerProperties.COMPILER_SUBPLAN_MERGE_KEY,
+                CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY, CompilerProperties.COMPILER_ARRAYINDEX_KEY,
+                CompilerProperties.COMPILER_CBO_KEY, CompilerProperties.COMPILER_CBO_TEST_KEY,
+                CompilerProperties.COMPILER_FORCE_JOIN_ORDER_KEY, CompilerProperties.COMPILER_QUERY_PLAN_SHAPE_KEY,
+                CompilerProperties.COMPILER_MIN_MEMORY_ALLOCATION_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
+                FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
+                StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
+                FeedActivityDetails.COLLECT_LOCATIONS, SqlppQueryRewriter.INLINE_WITH_OPTION,
+                SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION, "hash_merge", "output-record-type",
+                DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION,
+                SetAsterixPhysicalOperatorsRule.REWRITE_ATTEMPT_BATCH_ASSIGN,
+                EquivalenceClassUtils.REWRITE_INTERNAL_QUERYUID_PK, SqlppQueryRewriter.SQL_COMPAT_OPTION));
+    }
 }
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 b1d0b47..b8685d5 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
@@ -26,7 +26,6 @@
 import org.apache.asterix.external.library.ExternalFunctionDescriptorProvider;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IExternalFunctionInfo;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionManager;
 import org.apache.asterix.om.functions.IFunctionTypeInferer;
@@ -138,7 +137,7 @@
             throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
         IFunctionDescriptor fd;
-        if (expr.getFunctionInfo() instanceof IExternalFunctionInfo) {
+        if (expr.getFunctionInfo().isExternal()) {
             // Expr is an external function
             fd = ExternalFunctionDescriptorProvider.resolveExternalFunction(expr, env, context);
         } else {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
index 8b5c8bc..b6f79d9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
@@ -18,32 +18,70 @@
  */
 package org.apache.asterix.optimizer.base;
 
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
+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.common.metadata.DataverseName;
+import org.apache.asterix.common.transactions.TxnId;
+import org.apache.asterix.common.utils.JobUtils;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.metadata.declared.DataSourceId;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.declared.ResultSetDataSink;
+import org.apache.asterix.metadata.declared.ResultSetSinkId;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.optimizer.rules.am.AccessMethodUtils;
+import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
+import org.apache.asterix.translator.ResultMetadata;
+import org.apache.asterix.translator.SessionConfig;
 import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.compiler.api.ICompiler;
+import org.apache.hyracks.algebricks.compiler.api.ICompilerFactory;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+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.IVariableTypeEnvironment;
+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.operators.logical.AbstractDataSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
+import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.result.IResultSetReader;
+import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.control.nc.resources.memory.FrameManager;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleInputStream;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 
 public class AnalysisUtil {
+
+    private static final List<FunctionIdentifier> fieldAccessFunctions =
+            Arrays.asList(BuiltinFunctions.GET_DATA, BuiltinFunctions.GET_HANDLE, BuiltinFunctions.TYPE_OF);
+
     /*
      * If the first child of op is of type opType, then it returns that child,
      * o/w returns null.
@@ -204,11 +242,86 @@
         return true;
     }
 
-    private static List<FunctionIdentifier> fieldAccessFunctions = new ArrayList<>();
+    public static List<List<IAObject>> runQuery(Mutable<ILogicalOperator> topOp, List<LogicalVariable> resultVars,
+            IOptimizationContext queryOptCtx, IRuleSetKind ruleSetKind) throws AlgebricksException {
 
-    static {
-        fieldAccessFunctions.add(BuiltinFunctions.GET_DATA);
-        fieldAccessFunctions.add(BuiltinFunctions.GET_HANDLE);
-        fieldAccessFunctions.add(BuiltinFunctions.TYPE_OF);
+        MetadataProvider metadataProvider = (MetadataProvider) queryOptCtx.getMetadataProvider();
+        ICcApplicationContext appCtx = metadataProvider.getApplicationContext();
+        TxnId mainTxnId = metadataProvider.getTxnId();
+        try {
+            TxnId newTxnId = metadataProvider.getTxnIdFactory().create();
+            metadataProvider.setTxnId(newTxnId);
+
+            IVariableTypeEnvironment topOpTypeEnv = queryOptCtx.getOutputTypeEnvironment(topOp.getValue());
+            SerializerDeserializerProvider serdeProvider = SerializerDeserializerProvider.INSTANCE;
+
+            int nFields = resultVars.size();
+            List<Mutable<ILogicalExpression>> resultExprList = new ArrayList<>(nFields);
+            List<ISerializerDeserializer<?>> resultSerdeList = new ArrayList<>(nFields);
+
+            for (LogicalVariable var : resultVars) {
+                Object varType = topOpTypeEnv.getVarType(var);
+                if (varType == null) {
+                    throw new IllegalArgumentException("Cannot determine type of " + var);
+                }
+                resultSerdeList.add(serdeProvider.getSerializerDeserializer(varType));
+                resultExprList.add(new MutableObject<>(new VariableReferenceExpression(var)));
+            }
+
+            ResultMetadata resultMetadata = new ResultMetadata(SessionConfig.OutputFormat.ADM);
+            ResultSetId resultSetId = new ResultSetId(metadataProvider.getResultSetIdCounter().getAndInc());
+            ResultSetSinkId rssId = new ResultSetSinkId(resultSetId);
+            ResultSetDataSink sink = new ResultSetDataSink(rssId, null);
+
+            DistributeResultOperator resultOp = new DistributeResultOperator(resultExprList, sink, resultMetadata);
+            resultOp.getInputs().add(topOp);
+            queryOptCtx.computeAndSetTypeEnvironmentForOperator(resultOp);
+
+            MutableObject<ILogicalOperator> newResultOpRef = new MutableObject<>(resultOp);
+
+            ICompilerFactory compilerFactory = (ICompilerFactory) queryOptCtx.getCompilerFactory();
+            ICompiler compiler =
+                    compilerFactory.createCompiler(new ALogicalPlanImpl(newResultOpRef), queryOptCtx, ruleSetKind);
+            compiler.optimize();
+
+            JobSpecification jobSpec = compiler.createJob(appCtx, new JobEventListenerFactory(newTxnId, false));
+
+            JobId jobId = JobUtils.runJob(appCtx.getHcc(), jobSpec, true);
+
+            IResultSetReader resultSetReader = appCtx.getResultSet().createReader(jobId, resultSetId);
+            FrameManager frameManager = new FrameManager(queryOptCtx.getPhysicalOptimizationConfig().getFrameSize());
+            IFrame frame = new VSizeFrame(frameManager);
+
+            FrameTupleAccessor fta = new FrameTupleAccessor(null);
+            ByteArrayAccessibleInputStream bais = new ByteArrayAccessibleInputStream(frame.getBuffer().array(), 0, 0);
+            DataInputStream dis = new DataInputStream(bais);
+            List<List<IAObject>> result = new ArrayList<>();
+
+            while (resultSetReader.read(frame) > 0) {
+                ByteBuffer buffer = frame.getBuffer();
+                fta.reset(buffer);
+                int nTuples = fta.getTupleCount();
+                for (int tupleIdx = 0; tupleIdx < nTuples; tupleIdx++) {
+                    int tupleStart = fta.getTupleStartOffset(tupleIdx);
+                    int tupleEnd = fta.getTupleEndOffset(tupleIdx);
+                    bais.setContent(buffer.array(), tupleStart, tupleEnd - tupleStart);
+
+                    List<IAObject> values = new ArrayList<>(nFields);
+                    for (int fieldIdx = 0; fieldIdx < nFields; fieldIdx++) {
+                        IAObject value = (IAObject) resultSerdeList.get(fieldIdx).deserialize(dis);
+                        values.add(value);
+                    }
+                    result.add(values);
+                }
+            }
+
+            return result;
+        } catch (AlgebricksException e) {
+            throw e;
+        } catch (Exception e) {
+            throw new AlgebricksException(e);
+        } finally {
+            metadataProvider.setTxnId(mainTxnId);
+        }
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java
index 51c03eb..7b2dd7a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java
@@ -31,6 +31,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.IMissableTypeComputer;
 import org.apache.hyracks.algebricks.core.algebra.prettyprint.IPlanPrettyPrinter;
 import org.apache.hyracks.algebricks.core.rewriter.base.AlgebricksOptimizationContext;
+import org.apache.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 
@@ -39,17 +40,24 @@
 
 public final class AsterixOptimizationContext extends AlgebricksOptimizationContext {
 
-    private final Int2ObjectMap<Set<DataSource>> dataSourceMap = new Int2ObjectOpenHashMap<>();
+    private final Int2ObjectOpenHashMap<Set<DataSource>> dataSourceMap;
 
-    public AsterixOptimizationContext(int varCounter, IExpressionEvalSizeComputer expressionEvalSizeComputer,
+    public AsterixOptimizationContext(IOptimizationContextFactory optContextFactory, int varCounter,
+            IExpressionEvalSizeComputer expressionEvalSizeComputer,
             IMergeAggregationExpressionFactory mergeAggregationExpressionFactory,
             IExpressionTypeComputer expressionTypeComputer, IMissableTypeComputer nullableTypeComputer,
             IConflictingTypeResolver conflictingTypeResovler, PhysicalOptimizationConfig physicalOptimizationConfig,
             AlgebricksPartitionConstraint clusterLocations, IPlanPrettyPrinter prettyPrinter,
             IWarningCollector warningCollector) {
-        super(varCounter, expressionEvalSizeComputer, mergeAggregationExpressionFactory, expressionTypeComputer,
-                nullableTypeComputer, conflictingTypeResovler, physicalOptimizationConfig, clusterLocations,
-                prettyPrinter, warningCollector);
+        super(optContextFactory, varCounter, expressionEvalSizeComputer, mergeAggregationExpressionFactory,
+                expressionTypeComputer, nullableTypeComputer, conflictingTypeResovler, physicalOptimizationConfig,
+                clusterLocations, prettyPrinter, warningCollector);
+        dataSourceMap = new Int2ObjectOpenHashMap<>();
+    }
+
+    public AsterixOptimizationContext(AsterixOptimizationContext from) {
+        super(from);
+        dataSourceMap = from.dataSourceMap.clone();
     }
 
     public void addDataSource(DataSource dataSource) {
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 5493dd1..bebc4f2 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
@@ -26,6 +26,7 @@
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.optimizer.rules.AddEquivalenceClassForRecordConstructorRule;
+import org.apache.asterix.optimizer.rules.AnnotateOperatorCostCardinalityRule;
 import org.apache.asterix.optimizer.rules.AsterixConsolidateWindowOperatorsRule;
 import org.apache.asterix.optimizer.rules.AsterixExtractFunctionsFromJoinConditionRule;
 import org.apache.asterix.optimizer.rules.AsterixInlineVariablesRule;
@@ -96,6 +97,8 @@
 import org.apache.asterix.optimizer.rules.am.IntroduceLSMComponentFilterRule;
 import org.apache.asterix.optimizer.rules.am.IntroducePrimaryIndexForAggregationRule;
 import org.apache.asterix.optimizer.rules.am.IntroduceSelectAccessMethodRule;
+import org.apache.asterix.optimizer.rules.cbo.EnumerateJoinsRule;
+import org.apache.asterix.optimizer.rules.cbo.JoinEnum;
 import org.apache.asterix.optimizer.rules.subplan.AsterixMoveFreeVariableOperatorOutOfSubplanRule;
 import org.apache.asterix.optimizer.rules.subplan.InlineSubplanInputForNestedTupleSourceRule;
 import org.apache.asterix.optimizer.rules.temporal.TranslateIntervalExpressionRule;
@@ -354,6 +357,15 @@
         return dataExchange;
     }
 
+    public static final List<IAlgebraicRewriteRule> buildCBORuleCollection() {
+        List<IAlgebraicRewriteRule> cbo = new LinkedList<>();
+        cbo.add(new ConsolidateSelectsRule());
+        cbo.add(new EnumerateJoinsRule(new JoinEnum()));
+        cbo.add(new ReinferAllTypesRule());
+        cbo.add(new AsterixExtractFunctionsFromJoinConditionRule());
+        return cbo;
+    }
+
     public static final List<IAlgebraicRewriteRule> buildPhysicalRewritesAllLevelsRuleCollection() {
         List<IAlgebraicRewriteRule> physicalRewritesAllLevels = new LinkedList<>();
         physicalRewritesAllLevels.add(new PullSelectOutOfEqJoin());
@@ -420,6 +432,7 @@
         prepareForJobGenRewrites.add(new SweepIllegalNonfunctionalFunctions());
         prepareForJobGenRewrites.add(new FixReplicateOperatorOutputsRule());
         prepareForJobGenRewrites.add(new PopulateResultMetadataRule());
+        prepareForJobGenRewrites.add(new AnnotateOperatorCostCardinalityRule());
         return prepareForJobGenRewrites;
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/Cost.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/Cost.java
new file mode 100644
index 0000000..5dda277
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/Cost.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.cost;
+
+public class Cost implements ICost {
+
+    public static final double MAX_CARD = 1.0e200;
+    protected static final int COST_EQ = 0;
+
+    private final double cost;
+
+    public Cost() {
+        this.cost = 0.0;
+    }
+
+    public Cost(double cost) {
+        this.cost = cost;
+    }
+
+    @Override
+    public ICost zeroCost() {
+        return new Cost();
+    }
+
+    @Override
+    public ICost maxCost() {
+        return new Cost(MAX_CARD);
+    }
+
+    @Override
+    public ICost costAdd(ICost cost) {
+        return new Cost(computeTotalCost() + cost.computeTotalCost());
+    }
+
+    @Override
+    public boolean costEQ(ICost cost) {
+        return compareTo(cost) == COST_EQ;
+    }
+
+    @Override
+    public boolean costLT(ICost cost) {
+        return compareTo(cost) < COST_EQ;
+    }
+
+    @Override
+    public boolean costGT(ICost cost) {
+        return compareTo(cost) > COST_EQ;
+    }
+
+    @Override
+    public boolean costLE(ICost cost) {
+        return compareTo(cost) <= COST_EQ;
+    }
+
+    @Override
+    public boolean costGE(ICost cost) {
+        return compareTo(cost) >= COST_EQ;
+    }
+
+    @Override
+    public double computeTotalCost() {
+        return cost;
+    }
+
+    @Override
+    public int compareTo(ICost cost) {
+        return Double.compare(computeTotalCost(), cost.computeTotalCost());
+    }
+
+    @Override
+    public String toString() {
+        return Double.toString(cost);
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/CostMethods.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/CostMethods.java
new file mode 100644
index 0000000..5dafa54
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/CostMethods.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.optimizer.cost;
+
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.optimizer.rules.cbo.JoinNode;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+
+public class CostMethods implements ICostMethods {
+
+    protected IOptimizationContext optCtx;
+    protected PhysicalOptimizationConfig physOptConfig;
+    protected long blockSize;
+    protected long DOP;
+    protected double maxMemorySize;
+
+    public CostMethods(IOptimizationContext context) {
+        optCtx = context;
+        physOptConfig = context.getPhysicalOptimizationConfig();
+        blockSize = getBufferCachePageSize();
+        DOP = getDOP();
+        maxMemorySize = getMaxMemorySize();
+    }
+
+    public long getBufferCacheSize() {
+        MetadataProvider metadataProvider = (MetadataProvider) optCtx.getMetadataProvider();
+        return metadataProvider.getStorageProperties().getBufferCacheSize();
+    }
+
+    public long getBufferCachePageSize() {
+        MetadataProvider metadataProvider = (MetadataProvider) optCtx.getMetadataProvider();
+        return metadataProvider.getStorageProperties().getBufferCachePageSize();
+    }
+
+    public long getDOP() {
+        return optCtx.getComputationNodeDomain().cardinality();
+    }
+
+    public double getMaxMemorySize() {
+        return physOptConfig.getMaxFramesForJoin() * physOptConfig.getFrameSize();
+    }
+
+    // These cost methods are very simple and rudimentary for now. These can be improved by asterixdb developers as needed.
+    public Cost costFullScan(JoinNode jn) {
+        return new Cost(jn.computeJoinCardinality());
+    }
+
+    public Cost costIndexScan(JoinNode jn) {
+        return new Cost(jn.computeJoinCardinality());
+    }
+
+    public Cost costHashJoin(JoinNode jn) {
+        JoinNode leftJn = jn.getLeftJn();
+        JoinNode rightJn = jn.getRightJn();
+        return new Cost(leftJn.computeJoinCardinality() + rightJn.computeJoinCardinality());
+    }
+
+    public Cost computeHJProbeExchangeCost(JoinNode jn) {
+        JoinNode leftJn = jn.getLeftJn();
+        return new Cost(leftJn.computeJoinCardinality());
+    }
+
+    public Cost computeHJBuildExchangeCost(JoinNode jn) {
+        JoinNode rightJn = jn.getRightJn();
+        return new Cost(rightJn.computeJoinCardinality());
+    }
+
+    public Cost costBroadcastHashJoin(JoinNode jn) {
+        JoinNode leftJn = jn.getLeftJn();
+        JoinNode rightJn = jn.getRightJn();
+        return new Cost(leftJn.computeJoinCardinality() + DOP * rightJn.computeJoinCardinality());
+    }
+
+    public Cost computeBHJBuildExchangeCost(JoinNode jn) {
+        JoinNode rightJn = jn.getRightJn();
+        return new Cost(DOP * rightJn.computeJoinCardinality());
+    }
+
+    public Cost costIndexNLJoin(JoinNode jn) {
+        JoinNode leftJn = jn.getLeftJn();
+        JoinNode rightJn = jn.getRightJn();
+        return new Cost(leftJn.computeJoinCardinality());
+    }
+
+    public Cost computeNLJOuterExchangeCost(JoinNode jn) {
+        JoinNode leftJn = jn.getLeftJn();
+        return new Cost(DOP * leftJn.computeJoinCardinality());
+    }
+
+    public Cost costCartesianProductJoin(JoinNode jn) {
+        JoinNode leftJn = jn.getLeftJn();
+        JoinNode rightJn = jn.getRightJn();
+        return new Cost(leftJn.computeJoinCardinality() * rightJn.computeJoinCardinality());
+    }
+
+    public Cost computeCPRightExchangeCost(JoinNode jn) {
+        JoinNode rightJn = jn.getRightJn();
+        return new Cost(DOP * rightJn.computeJoinCardinality());
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/ICost.java
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/ICost.java
index 6095b26..99c3362 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/ICost.java
@@ -17,8 +17,26 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+package org.apache.asterix.optimizer.cost;
 
-select value v from range(1,2) v where v > ?;
+public interface ICost extends Comparable<ICost> {
+
+    ICost zeroCost();
+
+    ICost maxCost();
+
+    ICost costAdd(ICost cost);
+
+    boolean costEQ(ICost cost);
+
+    boolean costLT(ICost cost);
+
+    boolean costGT(ICost cost);
+
+    boolean costLE(ICost cost);
+
+    boolean costGE(ICost cost);
+
+    double computeTotalCost();
+
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/ICostMethods.java
similarity index 65%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/ICostMethods.java
index 6095b26..47ef617 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/cost/ICostMethods.java
@@ -17,8 +17,20 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+package org.apache.asterix.optimizer.cost;
 
-select value v from range(1,2) v where v > ?;
+import org.apache.asterix.optimizer.rules.cbo.JoinNode;
+
+public interface ICostMethods {
+    Cost costFullScan(JoinNode jn);
+
+    Cost costIndexScan(JoinNode jn);
+
+    Cost costHashJoin(JoinNode currentJn);
+
+    Cost costBroadcastHashJoin(JoinNode currentJn);
+
+    Cost costIndexNLJoin(JoinNode currentJn);
+
+    Cost costCartesianProductJoin(JoinNode currentJn);
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AnnotateOperatorCostCardinalityRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AnnotateOperatorCostCardinalityRule.java
new file mode 100644
index 0000000..b253a1d
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AnnotateOperatorCostCardinalityRule.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.optimizer.rules;
+
+import org.apache.asterix.optimizer.rules.cbo.EstimatedCostComputationVisitor;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * This rule annotates the operators with their estimated cardinality and estimated cost.
+ */
+public class AnnotateOperatorCostCardinalityRule implements IAlgebraicRewriteRule {
+
+    private boolean hasApplied = false;
+
+    public AnnotateOperatorCostCardinalityRule() {
+    }
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        if (hasApplied) {
+            return false;
+        }
+        hasApplied = true;
+        EstimatedCostComputationVisitor estCostCompVisitor = new EstimatedCostComputationVisitor();
+        opRef.getValue().accept(estCostCompVisitor, null);
+        return true;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        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 10aedc7..7202f74 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
@@ -29,7 +29,6 @@
 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.common.exceptions.NoOpWarningCollector;
 import org.apache.asterix.common.exceptions.WarningCollector;
 import org.apache.asterix.dataflow.data.common.ExpressionTypeComputer;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
@@ -48,7 +47,6 @@
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IExternalFunctionInfo;
 import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
@@ -90,11 +88,14 @@
 import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.serializer.ResultSerializerFactoryProvider;
+import org.apache.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
 import org.apache.hyracks.api.application.IServiceContext;
 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.IWarningCollector;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
 import org.apache.hyracks.api.exceptions.Warning;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -151,8 +152,9 @@
         jobGenCtx = new JobGenContext(null, metadataProvider, appCtx, SerializerDeserializerProvider.INSTANCE,
                 BinaryHashFunctionFactoryProvider.INSTANCE, BinaryHashFunctionFamilyProvider.INSTANCE,
                 BinaryComparatorFactoryProvider.INSTANCE, TypeTraitProvider.INSTANCE, BinaryBooleanInspector.FACTORY,
-                BinaryIntegerInspector.FACTORY, ADMPrinterFactoryProvider.INSTANCE, MissingWriterFactory.INSTANCE,
-                NullWriterFactory.INSTANCE, UnnestingPositionWriterFactory.INSTANCE, null,
+                BinaryIntegerInspector.FACTORY, ADMPrinterFactoryProvider.INSTANCE, PrinterBasedWriterFactory.INSTANCE,
+                ResultSerializerFactoryProvider.INSTANCE, MissingWriterFactory.INSTANCE, NullWriterFactory.INSTANCE,
+                UnnestingPositionWriterFactory.INSTANCE, null,
                 new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())),
                 ExpressionTypeComputer.INSTANCE, null, null, null, null, GlobalConfig.DEFAULT_FRAME_SIZE, null,
                 NoOpWarningCollector.INSTANCE, 0, new PhysicalOptimizationConfig());
@@ -390,7 +392,7 @@
         private boolean canConstantFold(ScalarFunctionCallExpression function) throws AlgebricksException {
             // skip external functions because they're not available at compile time (on CC)
             IFunctionInfo fi = function.getFunctionInfo();
-            if (fi instanceof IExternalFunctionInfo) {
+            if (fi.isExternal()) {
                 return false;
             }
             // skip all functions that would produce records/arrays/multisets (derived types) in their open format
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 3fc178a..09ad4d1 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,7 @@
 import java.util.Objects;
 import java.util.Set;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 import org.apache.asterix.algebra.operators.CommitOperator;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -190,18 +191,21 @@
             metaType = (ARecordType) mp.findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
         }
         List<Index> indexes = mp.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+        Stream<Index> indexStream = indexes.stream();
+        indexStream = indexStream.filter(index -> index.getIndexType() != IndexType.SAMPLE);
         if (primaryIndexModificationOp.getOperation() == Kind.INSERT && !primaryIndexModificationOp.isBulkload()) {
             // for insert, primary key index is handled together when primary index
-            indexes = indexes.stream().filter(index -> !index.isPrimaryKeyIndex()).collect(Collectors.toList());
+            indexStream = indexStream.filter(index -> !index.isPrimaryKeyIndex());
         }
-
-        // Set the top operator pointer to the primary IndexInsertDeleteOperator
-        ILogicalOperator currentTop = primaryIndexModificationOp;
+        indexes = indexStream.collect(Collectors.toList());
 
         // Put an n-gram or a keyword index in the later stage of index-update,
         // since TokenizeOperator needs to be involved.
         Collections.sort(indexes, (o1, o2) -> o1.getIndexType().ordinal() - o2.getIndexType().ordinal());
 
+        // Set the top operator pointer to the primary IndexInsertDeleteOperator
+        ILogicalOperator currentTop = primaryIndexModificationOp;
+
         // At this point, we have the data type info, and the indexes info as well
         int secondaryIndexTotalCnt = indexes.size() - 1;
         if (secondaryIndexTotalCnt > 0) {
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 aa2b8fd..036c456 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
@@ -95,7 +95,8 @@
             if (dataSource.getDatasourceType() != DataSource.Type.EXTERNAL_DATASET
                     && dataSource.getDatasourceType() != DataSource.Type.INTERNAL_DATASET
                     && dataSource.getDatasourceType() != DataSource.Type.LOADABLE
-                    && dataSource.getDatasourceType() != DataSource.Type.FUNCTION) {
+                    && dataSource.getDatasourceType() != DataSource.Type.FUNCTION
+                    && dataSource.getDatasourceType() != DataSource.Type.SAMPLE) {
                 IMutationDataSource mds = (IMutationDataSource) dataSource;
                 if (mds.isChange()) {
                     transformers = new ArrayList<>();
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 7f1ff4a..52f0279 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
@@ -120,11 +120,10 @@
      * @param analyzedAMs
      * @param context
      * @param isJoinLeftBranch
-     * @param isArbitraryFormOfSubtree
-     *            if the given subtree is in an arbitrary form that OptimizableSubTree class can't initialize, we try
-     *            to fill the field type of each variable that is used in the optimizable function expressions.
-     *            This way, an index-nested-loop-join transformation can be conducted properly since the transformation
-     *            process skips an optimzable function expression if the field type of one of its variable is unknown.
+     * @param isArbitraryFormOfSubtree if the given subtree is in an arbitrary form that OptimizableSubTree class can't initialize, we try
+     *                                 to fill the field type of each variable that is used in the optimizable function expressions.
+     *                                 This way, an index-nested-loop-join transformation can be conducted properly since the transformation
+     *                                 process skips an optimzable function expression if the field type of one of its variable is unknown.
      * @throws AlgebricksException
      */
     protected void fillSubTreeIndexExprs(OptimizableOperatorSubTree subTree,
@@ -174,13 +173,14 @@
     }
 
     protected void pruneIndexCandidates(Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs,
-            IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
+            IOptimizationContext context, IVariableTypeEnvironment typeEnvironment, boolean checkApplicableOnly)
+            throws AlgebricksException {
         Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
         // Check applicability of indexes by access method type.
         while (amIt.hasNext()) {
             Map.Entry<IAccessMethod, AccessMethodAnalysisContext> entry = amIt.next();
             AccessMethodAnalysisContext amCtx = entry.getValue();
-            pruneIndexCandidates(entry.getKey(), amCtx, context, typeEnvironment);
+            pruneIndexCandidates(entry.getKey(), amCtx, context, typeEnvironment, checkApplicableOnly);
             // Remove access methods for which there are definitely no
             // applicable indexes.
             if (amCtx.isIndexExprsAndVarsEmpty()) {
@@ -194,7 +194,8 @@
      * process by making it more systematic.
      */
     protected Pair<IAccessMethod, Index> chooseBestIndex(Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
-        List<Pair<IAccessMethod, Index>> list = chooseAllIndexes(analyzedAMs);
+        List<Pair<IAccessMethod, Index>> list = new ArrayList<>();
+        chooseAllIndexes(analyzedAMs, list);
         return list.isEmpty() ? null : list.get(0);
     }
 
@@ -206,9 +207,8 @@
      * [InvertedIndexAccessMethod, IndexType.SINGLE_PARTITION_WORD_INVIX || SINGLE_PARTITION_NGRAM_INVIX ||
      * LENGTH_PARTITIONED_WORD_INVIX || LENGTH_PARTITIONED_NGRAM_INVIX]
      */
-    protected List<Pair<IAccessMethod, Index>> chooseAllIndexes(
-            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
-        List<Pair<IAccessMethod, Index>> result = new ArrayList<>();
+    protected void chooseAllIndexes(Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs,
+            List<Pair<IAccessMethod, Index>> result) {
         // Use variables (fields) to the index types map to check which type of indexes are applied for the vars.
         Map<List<Pair<Integer, Integer>>, List<IndexType>> resultVarsToIndexTypesMap = new HashMap<>();
         Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
@@ -265,7 +265,6 @@
                 }
             }
         }
-        return result;
     }
 
     private boolean isSameFullTextConfigInIndexAndQuery(AccessMethodAnalysisContext analysisCtx,
@@ -305,7 +304,8 @@
      * @throws AlgebricksException
      */
     public void pruneIndexCandidates(IAccessMethod accessMethod, AccessMethodAnalysisContext analysisCtx,
-            IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
+            IOptimizationContext context, IVariableTypeEnvironment typeEnvironment, boolean checkApplicableOnly)
+            throws AlgebricksException {
         Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexExprAndVarIt =
                 analysisCtx.getIteratorForIndexExprsAndVars();
         boolean hasIndexPreferences = false;
@@ -363,7 +363,7 @@
                     final IOptimizableFuncExpr optFuncExpr = analysisCtx.getMatchedFuncExpr(exprAndVarIdx.first);
                     // If expr is not optimizable by concrete index then remove
                     // expr and continue.
-                    if (!accessMethod.exprIsOptimizable(index, optFuncExpr)) {
+                    if (!accessMethod.exprIsOptimizable(index, optFuncExpr, checkApplicableOnly)) {
                         exprsAndVarIter.remove();
                         continue;
                     }
@@ -662,7 +662,7 @@
      * optimizable function expression.
      *
      * @return true if a candidate index was added to foundIndexExprs, false
-     *         otherwise
+     * otherwise
      * @throws AlgebricksException
      */
     protected boolean fillIndexExprs(List<Index> datasetIndexes, List<String> fieldName, IAType fieldType,
@@ -821,7 +821,7 @@
         // Remember matching subtree.
         optFuncExpr.setOptimizableSubTree(funcVarIndex, subTree);
         List<String> fieldName = null;
-        MutableInt fieldSource = new MutableInt(0);
+        int fieldSource = 0;
         if (subTree.getDataSourceType() == DataSourceType.COLLECTION_SCAN) {
             ILogicalExpression expr = optFuncExpr.getArgument(funcVarIndex).getValue();
             if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
@@ -832,9 +832,11 @@
             if (subTree.getDataSourceType() == DataSourceType.DATASOURCE_SCAN) {
                 subTree.setLastMatchedDataSourceVars(0, funcVarIndex);
             }
-            fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0,
-                    subTree.getRecordType(), funcVarIndex, optFuncExpr.getArgument(funcVarIndex).getValue(),
-                    subTree.getMetaRecordType(), datasetMetaVar, fieldSource, false);
+            Pair<List<String>, Integer> fieldNameAndSource =
+                    AccessMethodUtils.getFieldNameSetStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0,
+                            funcVarIndex, optFuncExpr.getArgument(funcVarIndex).getValue(), context);
+            fieldName = fieldNameAndSource.first;
+            fieldSource = fieldNameAndSource.second;
             if (fieldName.isEmpty()) {
                 return;
             }
@@ -843,13 +845,13 @@
                 (IAType) context.getOutputTypeEnvironment(unnestOp).getType(optFuncExpr.getLogicalExpr(funcVarIndex));
         // Set the fieldName in the corresponding matched function
         // expression.
-        optFuncExpr.setFieldName(funcVarIndex, fieldName, fieldSource.intValue());
+        optFuncExpr.setFieldName(funcVarIndex, fieldName, fieldSource);
         optFuncExpr.setFieldType(funcVarIndex, fieldType);
 
         setTypeTag(context, subTree, optFuncExpr, funcVarIndex);
         if (subTree.hasDataSource()) {
             fillIndexExprs(datasetIndexes, fieldName, fieldType, optFuncExpr, optFuncExprIndex, funcVarIndex, subTree,
-                    analysisCtx, fieldSource.intValue(), accessMethod);
+                    analysisCtx, fieldSource, accessMethod);
         }
     }
 
@@ -860,7 +862,6 @@
         boolean doesArrayIndexQualify = context.getPhysicalOptimizationConfig().isArrayIndexEnabled()
                 && datasetIndexes.stream().anyMatch(i -> i.getIndexType() == IndexType.ARRAY);
         List<LogicalVariable> varList = assignOp.getVariables();
-        MutableInt fieldSource = new MutableInt(0);
         for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
             LogicalVariable var = varList.get(varIndex);
             int optVarIndex = optFuncExpr.findLogicalVar(var);
@@ -871,8 +872,7 @@
                             .analyzeVarForArrayIndexes(datasetIndexes, optFuncExpr, subTree, context, var, analysisCtx);
                     if (fieldTriplet != null && subTree.hasDataSource()) {
                         fillIndexExprs(datasetIndexes, fieldTriplet.second, fieldTriplet.third, optFuncExpr,
-                                optFuncExprIndex, fieldTriplet.first, subTree, analysisCtx, fieldSource.intValue(),
-                                accessMethod);
+                                optFuncExprIndex, fieldTriplet.first, subTree, analysisCtx, 0, accessMethod);
                     }
                 }
                 continue;
@@ -885,22 +885,22 @@
                 subTree.setLastMatchedDataSourceVars(varIndex, optVarIndex);
             }
 
-            fieldSource.setValue(0);
-            List<String> fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(optFuncExpr, subTree,
-                    assignOrUnnestIndex, varIndex, subTree.getRecordType(), optVarIndex,
-                    optFuncExpr.getArgument(optVarIndex).getValue(), subTree.getMetaRecordType(), datasetMetaVar,
-                    fieldSource, false);
+            Pair<List<String>, Integer> fieldNameAndSource =
+                    AccessMethodUtils.getFieldNameSetStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex,
+                            varIndex, optVarIndex, optFuncExpr.getArgument(optVarIndex).getValue(), context);
+            List<String> fieldName = fieldNameAndSource.first;
+            int fieldSource = fieldNameAndSource.second;
 
             IAType fieldType = (IAType) context.getOutputTypeEnvironment(assignOp).getVarType(var);
             // Set the fieldName in the corresponding matched
             // function expression.
-            optFuncExpr.setFieldName(optVarIndex, fieldName, fieldSource.intValue());
+            optFuncExpr.setFieldName(optVarIndex, fieldName, fieldSource);
             optFuncExpr.setFieldType(optVarIndex, fieldType);
 
             setTypeTag(context, subTree, optFuncExpr, optVarIndex);
             if (subTree.hasDataSource()) {
                 fillIndexExprs(datasetIndexes, fieldName, fieldType, optFuncExpr, optFuncExprIndex, optVarIndex,
-                        subTree, analysisCtx, fieldSource.intValue(), accessMethod);
+                        subTree, analysisCtx, fieldSource, accessMethod);
             }
         }
     }
@@ -989,16 +989,8 @@
     /**
      * Finds the field name of each variable in the ASSIGN or UNNEST operators of the sub-tree.
      */
-    protected void fillFieldNamesInTheSubTree(OptimizableOperatorSubTree subTree) throws AlgebricksException {
-        LogicalVariable datasetMetaVar = null;
-        if (subTree.getDataSourceType() != DataSourceType.COLLECTION_SCAN
-                && subTree.getDataSourceType() != DataSourceType.INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP) {
-            List<LogicalVariable> datasetVars = subTree.getDataSourceVariables();
-            if (subTree.getDataset().hasMetaPart()) {
-                datasetMetaVar = datasetVars.get(datasetVars.size() - 1);
-            }
-        }
-        MutableInt fieldSource = new MutableInt(0);
+    protected void fillFieldNamesInTheSubTree(OptimizableOperatorSubTree subTree, IOptimizationContext context)
+            throws AlgebricksException {
         for (int assignOrUnnestIndex = 0; assignOrUnnestIndex < subTree.getAssignsAndUnnests()
                 .size(); assignOrUnnestIndex++) {
             AbstractLogicalOperator op = subTree.getAssignsAndUnnests().get(assignOrUnnestIndex);
@@ -1009,10 +1001,8 @@
                     LogicalVariable var = varList.get(varIndex);
                     // funcVarIndex is not required. Thus, we set it to -1.
                     // optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
-                    fieldSource.setValue(0);
                     List<String> fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(null, subTree,
-                            assignOrUnnestIndex, varIndex, subTree.getRecordType(), -1, null,
-                            subTree.getMetaRecordType(), datasetMetaVar, fieldSource, false);
+                            assignOrUnnestIndex, varIndex, -1, null, context).first;
                     if (fieldName != null && !fieldName.isEmpty()) {
                         subTree.getVarsToFieldNameMap().put(var, fieldName);
                     }
@@ -1020,14 +1010,11 @@
             } else if (op.getOperatorTag() == LogicalOperatorTag.UNNEST) {
                 UnnestOperator unnestOp = (UnnestOperator) op;
                 LogicalVariable var = unnestOp.getVariable();
-                List<String> fieldName = null;
                 if (subTree.getDataSourceType() != DataSourceType.COLLECTION_SCAN) {
                     // funcVarIndex is not required. Thus, we set it to -1.
                     // optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
-                    fieldSource.setValue(0);
-                    fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(null, subTree, assignOrUnnestIndex, 0,
-                            subTree.getRecordType(), -1, null, subTree.getMetaRecordType(), datasetMetaVar, fieldSource,
-                            false);
+                    List<String> fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(null, subTree,
+                            assignOrUnnestIndex, 0, -1, null, context).first;
                     if (fieldName != null && !fieldName.isEmpty()) {
                         subTree.getVarsToFieldNameMap().put(var, fieldName);
                     }
@@ -1052,10 +1039,8 @@
                     LogicalVariable var = varList.get(varIndex);
                     // funcVarIndex is not required. Thus, we set it to -1.
                     // optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
-                    fieldSource.setValue(0);
                     List<String> fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(null, subTree,
-                            assignOrUnnestIndex, varIndex, subTree.getRecordType(), -1, null,
-                            subTree.getMetaRecordType(), datasetMetaVar, fieldSource, false);
+                            assignOrUnnestIndex, varIndex, -1, null, context).first;
                     if (fieldName != null && !fieldName.isEmpty()) {
                         subTree.getVarsToFieldNameMap().put(var, fieldName);
                     }
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 3b39d9d..0196d91 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
@@ -24,7 +24,6 @@
 import static org.apache.asterix.om.functions.BuiltinFunctions.FIELD_ACCESS_NESTED;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
@@ -147,6 +146,8 @@
             BuiltinFunctions.YEAR_MONTH_DURATION_DEFAULT_NULL_CONSTRUCTOR,
             BuiltinFunctions.UUID_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.BINARY_BASE64_DEFAULT_NULL_CONSTRUCTOR);
 
+    private final static Pair<List<String>, Integer> NO_FIELD_NAME = new Pair<>(Collections.emptyList(), 0);
+
     public static void appendPrimaryIndexTypes(Dataset dataset, IAType itemType, IAType metaItemType,
             List<Object> target) throws AlgebricksException {
         ARecordType recordType = (ARecordType) itemType;
@@ -1735,10 +1736,10 @@
      * false positive results, we can apply instantTryLock() on PK optimization since a result from these indexes
      * doesn't have to be verified by the primary index-lookup and a subsequent SELECT operator.
      * (i.e., we can guarantee the correctness of the result.)
-     *
+     * <p>
      * Case A) non-index-only plan
      * sidx-search -> (optional) sort -> (optional) distinct -> pdix-search
-     *
+     * <p>
      * Case B) index-only plan
      * left path (an instantTryLock() on the PK fail path):
      * right path(an instantTryLock() on the PK success path):
@@ -2358,10 +2359,9 @@
      * unless the variables are produced after the SELECT (JOIN) operator.
      *
      * @return Pair<Boolean, Boolean>: the first boolean value tells whether the given plan is an index-only plan.
-     *         The second boolean value tells whether the secondary key field variable(s) are used after the given
-     *         SELECT (JOIN) operator.
+     * The second boolean value tells whether the secondary key field variable(s) are used after the given
+     * SELECT (JOIN) operator.
      * @throws AlgebricksException
-     *
      */
     private static void checkVarUsageAfterSelectOp(List<Mutable<ILogicalOperator>> afterSelectOpRefs,
             List<LogicalVariable> liveVarsAfterSelJoinOp, List<LogicalVariable> dataScanPKVars,
@@ -2848,12 +2848,9 @@
     /**
      * Checks whether a LogicalVariable exists in a list of Triple<LogicalVariable, LogicalVariable, LogicalVariable>.
      *
-     * @param varsList
-     *            list that contains triples of LogicalVariable.
-     * @param varToFind
-     *            a LogicalVariable to find
-     * @param checkOnlyFirst
-     *            specifies whether it is required to check only the first variable in the given triple.
+     * @param varsList       list that contains triples of LogicalVariable.
+     * @param varToFind      a LogicalVariable to find
+     * @param checkOnlyFirst specifies whether it is required to check only the first variable in the given triple.
      * @return
      */
     public static boolean findVarInTripleVarList(
@@ -2898,17 +2895,16 @@
         return ann == null ? null : ann.getIndexNames();
     }
 
-    public static List<String> getFieldNameSetStepsFromSubTree(IOptimizableFuncExpr optFuncExpr,
-            OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, ARecordType recordType,
-            int funcVarIndex, ILogicalExpression parentFuncExpr, ARecordType metaType, LogicalVariable metaVar,
-            MutableInt fieldSource, boolean isUnnestOverVarAllowed) throws AlgebricksException {
+    public static Pair<List<String>, Integer> getFieldNameSetStepsFromSubTree(IOptimizableFuncExpr optFuncExpr,
+            OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, int funcVarIndex,
+            ILogicalExpression parentFuncExpr, IOptimizationContext context) throws AlgebricksException {
         if (optFuncExpr != null) {
             if (parentFuncExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                 optFuncExpr.addStepExpr(funcVarIndex, ((AbstractFunctionCallExpression) parentFuncExpr));
             }
         }
-        return getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, opIndex, assignVarIndex, recordType, funcVarIndex,
-                parentFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+        return getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, opIndex, assignVarIndex, funcVarIndex,
+                parentFuncExpr, context);
     }
 
     /**
@@ -2918,10 +2914,9 @@
      *
      * @throws AlgebricksException
      */
-    private static List<String> getFieldNameAndStepsFromSubTree(IOptimizableFuncExpr optFuncExpr,
-            OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, ARecordType recordType,
-            int funcVarIndex, ILogicalExpression parentFuncExpr, ARecordType metaType, LogicalVariable metaVar,
-            MutableInt fieldSource, boolean isUnnestOverVarAllowed) throws AlgebricksException {
+    private static Pair<List<String>, Integer> getFieldNameAndStepsFromSubTree(IOptimizableFuncExpr optFuncExpr,
+            OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, int funcVarIndex,
+            ILogicalExpression parentFuncExpr, IOptimizationContext context) throws AlgebricksException {
         // Get expression corresponding to opVar at varIndex.
         AbstractLogicalExpression expr = null;
         AbstractFunctionCallExpression childFuncExpr = null;
@@ -2931,23 +2926,23 @@
             expr = (AbstractLogicalExpression) assignOp.getExpressions().get(assignVarIndex).getValue();
             // Can't get a field name from a constant expression. So, return null.
             if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-                return Collections.emptyList();
+                return NO_FIELD_NAME;
             }
             childFuncExpr = (AbstractFunctionCallExpression) expr;
         } else {
             UnnestOperator unnestOp = (UnnestOperator) op;
             expr = (AbstractLogicalExpression) unnestOp.getExpressionRef().getValue();
             if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-                return Collections.emptyList();
+                return NO_FIELD_NAME;
             }
             childFuncExpr = (AbstractFunctionCallExpression) expr;
             if (childFuncExpr.getFunctionIdentifier() != BuiltinFunctions.SCAN_COLLECTION) {
-                return Collections.emptyList();
+                return NO_FIELD_NAME;
             }
             expr = (AbstractLogicalExpression) childFuncExpr.getArguments().get(0).getValue();
         }
         if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-            return Collections.emptyList();
+            return NO_FIELD_NAME;
         }
         AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
         FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
@@ -2960,21 +2955,21 @@
         if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_NAME) {
             fieldName = ConstantExpressionUtil.getStringArgument(funcExpr, 1);
             if (fieldName == null) {
-                return Collections.emptyList();
+                return NO_FIELD_NAME;
             }
             isFieldAccess = true;
             isByName = true;
         } else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_INDEX) {
             Integer idx = ConstantExpressionUtil.getIntArgument(funcExpr, 1);
             if (idx == null) {
-                return Collections.emptyList();
+                return NO_FIELD_NAME;
             }
             fieldIndex = idx;
             isFieldAccess = true;
         } else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_NESTED) {
             ILogicalExpression nameArg = funcExpr.getArguments().get(1).getValue();
             if (nameArg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                return Collections.emptyList();
+                return NO_FIELD_NAME;
             }
             ConstantExpression constExpr = (ConstantExpression) nameArg;
             AOrderedList orderedNestedFieldName =
@@ -2989,11 +2984,6 @@
         if (isFieldAccess) {
             LogicalVariable sourceVar =
                     ((VariableReferenceExpression) funcExpr.getArguments().get(0).getValue()).getVariableReference();
-            if (sourceVar.equals(metaVar)) {
-                fieldSource.setValue(1);
-            } else {
-                fieldSource.setValue(0);
-            }
             if (optFuncExpr != null) {
                 optFuncExpr.setLogicalExpr(funcVarIndex, parentFuncExpr);
                 optFuncExpr.addStepExpr(funcVarIndex, funcExpr);
@@ -3024,8 +3014,7 @@
                     if (parentVars.contains(var)) {
                         //Found the variable we are looking for.
                         //return assign and index of expression
-                        int[] returnValues = { i, varIndex };
-                        assignAndExpressionIndexes = returnValues;
+                        assignAndExpressionIndexes = new int[] { i, varIndex };
                     }
                 }
             }
@@ -3033,34 +3022,25 @@
                 //We found the nested assign
 
                 //Recursive call on nested assign
-                List<String> parentFieldNames = getFieldNameAndStepsFromSubTree(optFuncExpr, subTree,
-                        assignAndExpressionIndexes[0], assignAndExpressionIndexes[1], recordType, funcVarIndex,
-                        parentFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+                Pair<List<String>, Integer> parentFieldNames =
+                        getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignAndExpressionIndexes[0],
+                                assignAndExpressionIndexes[1], funcVarIndex, parentFuncExpr, context);
 
-                boolean isPreviousOperatorLegalUnnest = isUnnestOverVarAllowed && subTree.getAssignsAndUnnests()
-                        .get(assignAndExpressionIndexes[0]).getOperatorTag().equals(LogicalOperatorTag.UNNEST);
-                if (parentFieldNames.isEmpty() && !isPreviousOperatorLegalUnnest) {
+                if (parentFieldNames.first.isEmpty()) {
                     //Nested assign was not a field access.
                     //We will not use index
-                    return Collections.emptyList();
-                } else if (isPreviousOperatorLegalUnnest) {
-                    parentFieldNames = new ArrayList<>();
+                    return NO_FIELD_NAME;
                 }
 
                 if (!isByName) {
-                    IAType subFieldType;
-                    if (isUnnestOverVarAllowed && isPreviousOperatorLegalUnnest) {
-                        // In the case of UNNESTing over a variable, we use the record type given by our caller instead.
-                        subFieldType = sourceVar.equals(metaVar) ? metaType : recordType;
-                    } else {
-                        subFieldType = sourceVar.equals(metaVar) ? metaType.getSubFieldType(parentFieldNames)
-                                : recordType.getSubFieldType(parentFieldNames);
-                        // Sub-field type can be AUnionType in case if optional. Thus, needs to get the actual type.
-                        subFieldType = TypeComputeUtils.getActualType(subFieldType);
-                        if (subFieldType.getTypeTag() != ATypeTag.OBJECT) {
-                            throw CompilationException.create(ErrorCode.TYPE_CONVERT, subFieldType,
-                                    ARecordType.class.getName());
-                        }
+                    IVariableTypeEnvironment outputTypeEnvironment = context.getOutputTypeEnvironment(
+                            subTree.getAssignsAndUnnests().get(assignAndExpressionIndexes[0]));
+                    IAType subFieldType = (IAType) outputTypeEnvironment.getVarType(sourceVar);
+                    // Sub-field type can be AUnionType in case if optional. Thus, needs to get the actual type.
+                    subFieldType = TypeComputeUtils.getActualType(subFieldType);
+                    if (subFieldType.getTypeTag() != ATypeTag.OBJECT) {
+                        throw CompilationException.create(ErrorCode.TYPE_CONVERT, subFieldType,
+                                ARecordType.class.getName());
                     }
                     fieldName = ((ARecordType) subFieldType).getFieldNames()[fieldIndex];
 
@@ -3070,11 +3050,9 @@
                 }
                 //add fieldName to the nested fieldName, return
                 if (nestedAccessFieldName != null) {
-                    for (int i = 0; i < nestedAccessFieldName.size(); i++) {
-                        parentFieldNames.add(nestedAccessFieldName.get(i));
-                    }
+                    parentFieldNames.first.addAll(nestedAccessFieldName);
                 } else {
-                    parentFieldNames.add(fieldName);
+                    parentFieldNames.first.add(fieldName);
                 }
                 return (parentFieldNames);
             }
@@ -3083,15 +3061,15 @@
                 optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
             }
             //no nested assign, we are at the lowest level.
+            OptimizableOperatorSubTree.RecordTypeSource recType = subTree.getRecordTypeFor(sourceVar);
             if (isByName) {
                 if (nestedAccessFieldName != null) {
-                    return nestedAccessFieldName;
+                    return new Pair<>(nestedAccessFieldName, recType.sourceIndicator);
                 }
-                return new ArrayList<>(Arrays.asList(fieldName));
+                return new Pair<>(new ArrayList<>(List.of(fieldName)), recType.sourceIndicator);
             }
-            return new ArrayList<>(Arrays.asList(sourceVar.equals(metaVar) ? metaType.getFieldNames()[fieldIndex]
-                    : recordType.getFieldNames()[fieldIndex]));
-
+            return new Pair<>(new ArrayList<>(List.of(recType.recordType.getFieldNames()[fieldIndex])),
+                    recType.sourceIndicator);
         }
 
         // We use a part of the field in edit distance computation
@@ -3101,17 +3079,17 @@
         }
         List<Mutable<ILogicalExpression>> funcArgs = funcExpr.getArguments();
         if (funcArgs.isEmpty()) {
-            return Collections.emptyList();
+            return NO_FIELD_NAME;
         }
         // We expect the function's argument to be a variable, otherwise we
         // cannot apply an index.
         ILogicalExpression argExpr = funcArgs.get(0).getValue();
         if (argExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
-            return Collections.emptyList();
+            return NO_FIELD_NAME;
         }
         for (int i = 1; i < funcArgs.size(); i++) {
             if (funcArgs.get(i).getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                return Collections.emptyList();
+                return NO_FIELD_NAME;
             }
         }
         if (optFuncExpr != null) {
@@ -3131,20 +3109,19 @@
                     if (var.equals(curVar) && optFuncExpr != null) {
                         optFuncExpr.setSourceVar(funcVarIndex, var);
                         return getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, varIndex,
-                                recordType, funcVarIndex, childFuncExpr, metaType, metaVar, fieldSource,
-                                isUnnestOverVarAllowed);
+                                funcVarIndex, childFuncExpr, context);
                     }
                 }
             } else {
                 UnnestOperator unnestOp = (UnnestOperator) curOp;
                 LogicalVariable var = unnestOp.getVariable();
                 if (var.equals(curVar)) {
-                    getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, recordType,
-                            funcVarIndex, childFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+                    getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, funcVarIndex,
+                            childFuncExpr, context);
                 }
             }
         }
-        return Collections.emptyList();
+        return NO_FIELD_NAME;
     }
 
     public static Triple<Integer, List<String>, IAType> analyzeVarForArrayIndexes(List<Index> datasetIndexes,
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 2eebe5b..7c09b59 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
@@ -1020,7 +1020,8 @@
     }
 
     @Override
-    public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) throws AlgebricksException {
+    public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr, boolean checkApplicableOnly)
+            throws AlgebricksException {
         // If we are optimizing a join, check for the indexed nested-loop join hint.
         if (optFuncExpr.getNumLogicalVars() == 2) {
             if (optFuncExpr.getOperatorSubTree(0) == optFuncExpr.getOperatorSubTree(1)) {
@@ -1033,7 +1034,8 @@
                     //And we were unable to determine its type
                     return false;
                 }
-            } else if (!optFuncExpr.getFuncExpr().hasAnnotation(IndexedNLJoinExpressionAnnotation.class)) {
+            } else if (!checkApplicableOnly
+                    && !optFuncExpr.getFuncExpr().hasAnnotation(IndexedNLJoinExpressionAnnotation.class)) {
                 return false;
             }
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
index 5688a70..bac1a0b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
@@ -47,8 +47,8 @@
 
     /**
      * @return A list of function identifiers that are optimizable by this
-     *         access method. Also, the second boolean tells whether that
-     *         function can generate a false-positive result.
+     * access method. Also, the second boolean tells whether that
+     * function can generate a false-positive result.
      */
     public List<Pair<FunctionIdentifier, Boolean>> getOptimizableFunctions();
 
@@ -61,7 +61,7 @@
      * OptimizableFunction to analysisCtx.matchedFuncExprs for further analysis.
      *
      * @return true if funcExpr is optimizable by this access method, false
-     *         otherwise
+     * otherwise
      * @throws AlgebricksException
      */
     boolean analyzeFuncExprArgsAndUpdateAnalysisCtx(AbstractFunctionCallExpression funcExpr,
@@ -71,8 +71,8 @@
     /**
      * Indicates whether this access method is applicable for the given index type.
      *
-     * @return boolean
      * @param indexType
+     * @return boolean
      */
     public boolean matchIndexType(IndexType indexType);
 
@@ -80,8 +80,8 @@
      * Indicates whether all index expressions must be matched in order for this
      * index to be applicable.
      *
-     * @return boolean
      * @param index
+     * @return boolean
      */
     public boolean matchAllIndexExprs(Index index);
 
@@ -89,8 +89,8 @@
      * Indicates whether this index is applicable if only a prefix of the index
      * expressions are matched.
      *
-     * @return boolean
      * @param index
+     * @return boolean
      */
     public boolean matchPrefixIndexExprs(Index index);
 
@@ -127,7 +127,8 @@
      *
      * @throws AlgebricksException
      */
-    public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) throws AlgebricksException;
+    public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr, boolean checkApplicableOnly)
+            throws AlgebricksException;
 
     public Collection<String> getSecondaryIndexPreferences(IOptimizableFuncExpr optFuncExpr);
 
@@ -136,12 +137,11 @@
     /**
      * Checks whether the function applied to an indexed field is acceptable by the access method.
      *
-     * @param functionExpr applied function
-     * @param index the index definition
+     * @param functionExpr     applied function
+     * @param index            the index definition
      * @param indexedFieldType the type of the indexed field in the index definition
-     * @param defaultNull true if the candidate index has CAST (DEFAULT NULL) modifier
-     * @param finalStep true if the functionExpr is the final function applied
-     *
+     * @param defaultNull      true if the candidate index has CAST (DEFAULT NULL) modifier
+     * @param finalStep        true if the functionExpr is the final function applied
      * @return true if the access method accepts the argument function. False, otherwise.
      */
     public boolean acceptsFunction(AbstractFunctionCallExpression functionExpr, Index index, IAType indexedFieldType,
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
index 6f53219..b2b5e7c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
@@ -58,21 +58,21 @@
  * The order of the join inputs matters (left branch - outer relation, right branch - inner relation).
  * This rule tries to utilize an index on the inner relation.
  * If that's not possible, it stops transforming the given join into an index-nested-loop join.
- *
+ * <p>
  * This rule replaces the above pattern with the following simplified plan:
  * select <-- assign+ <-- unnest-map(pidx) <-- sort <-- unnest-map(sidx) <-- assign+ <-- (datasource scan|unnest-map)
  * The sorting PK process is optional, and some access methods may choose not to sort.
  * Note that for some index-based optimizations we do not remove the triggering
  * condition from the join, since the secondary index may only act as a filter, and the
  * final verification must still be done with the original join condition.
- *
+ * <p>
  * The basic outline of this rule is:
  * 1. Match operator pattern.
  * 2. Analyze join condition to see if there are optimizable functions (delegated to IAccessMethods).
  * 3. Check metadata to see if there are applicable indexes.
  * 4. Choose an index to apply (for now only a single index will be chosen).
  * 5. Rewrite plan using index (delegated to IAccessMethods).
- *
+ * <p>
  * For left-outer-join, additional patterns are checked and additional treatment is needed as follows:
  * 1. First it checks if there is a groupByOp above the join: groupby <-- leftouterjoin
  * 2. Inherently, only the right-subtree of the lojOp can be used as indexSubtree.
@@ -81,7 +81,7 @@
  * Here, the primary key variable from datasourceScanOp replaces the introduced null placeholder variable.
  * If the primary key is a composite key, then the first variable of the primary key variables becomes the
  * null place holder variable. This null placeholder variable works for all three types of indexes.
- *
+ * <p>
  * If the inner-branch can be transformed as an index-only plan, this rule creates an index-only-plan path
  * that is similar to one described in IntroduceSelectAccessMethod Rule.
  */
@@ -140,7 +140,7 @@
         afterJoinRefs = new ArrayList<>();
         // Recursively checks the given plan whether the desired pattern exists in it.
         // If so, try to optimize the plan.
-        boolean planTransformed = checkAndApplyJoinTransformation(opRef, context);
+        boolean planTransformed = checkAndApplyJoinTransformation(opRef, context, false);
 
         if (joinOp != null) {
             // We found an optimization here. Don't need to optimize this operator again.
@@ -156,6 +156,27 @@
         return planTransformed;
     }
 
+    public boolean checkApplicable(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        clear();
+        setMetadataDeclarations(context);
+
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+
+        afterJoinRefs = new ArrayList<>();
+        // Recursively checks the given plan whether the desired pattern exists in it.
+        // If so, try to optimize the plan.
+        boolean planTransformed = checkAndApplyJoinTransformation(opRef, context, true);
+
+        if (!planTransformed) {
+            return false;
+        } else {
+            OperatorPropertiesUtil.typeOpRec(opRef, context);
+        }
+
+        return planTransformed;
+    }
+
     /**
      * Removes indexes from the outer branch from the optimizer's consideration for this rule,
      * since we only use indexes from the inner branch.
@@ -234,8 +255,8 @@
      * optimize the path from the given join operator to the EMPTY_TUPLE_SOURCE operator
      * if it is not already optimized.
      */
-    protected boolean checkAndApplyJoinTransformation(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
-            throws AlgebricksException {
+    protected boolean checkAndApplyJoinTransformation(Mutable<ILogicalOperator> opRef, IOptimizationContext context,
+            boolean checkApplicableOnly) throws AlgebricksException {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
         boolean joinFoundAndOptimizationApplied;
 
@@ -246,7 +267,7 @@
         // Recursively check the plan and try to optimize it. We first check the children of the given operator
         // to make sure an earlier join in the path is optimized first.
         for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
-            joinFoundAndOptimizationApplied = checkAndApplyJoinTransformation(inputOpRef, context);
+            joinFoundAndOptimizationApplied = checkAndApplyJoinTransformation(inputOpRef, context, checkApplicableOnly);
             if (joinFoundAndOptimizationApplied) {
                 return true;
             }
@@ -313,7 +334,7 @@
                 // the subplan into the index branch and giving the join a condition for this rule to optimize.
                 // *No nodes* from this rewrite will be used beyond this point.
                 joinFromSubplanRewrite.findAfterSubplanSelectOperator(afterJoinRefs);
-                if (rewriteLocallyAndTransform(joinRef, context, joinFromSubplanRewrite)) {
+                if (rewriteLocallyAndTransform(joinRef, context, joinFromSubplanRewrite, checkApplicableOnly)) {
                     // Connect the after-join operators to the index subtree root before this rewrite. This also avoids
                     // performing the secondary index validation step twice.
                     ILogicalOperator lastAfterJoinOp = afterJoinRefs.get(afterJoinRefs.size() - 1).getValue();
@@ -364,7 +385,7 @@
                 fillSubTreeIndexExprs(rightSubTree, analyzedAMs, context, false);
 
                 // Prunes the access methods based on the function expression and access methods.
-                pruneIndexCandidates(analyzedAMs, context, typeEnvironment);
+                pruneIndexCandidates(analyzedAMs, context, typeEnvironment, checkApplicableOnly);
 
                 // If the right subtree (inner branch) has indexes, one of those indexes will be used.
                 // Removes the indexes from the outer branch in the optimizer's consideration list for this rule.
@@ -382,10 +403,10 @@
                     // Finds the field name of each variable in the sub-tree such as variables for order by.
                     // This step is required when checking index-only plan.
                     if (checkLeftSubTreeMetadata) {
-                        fillFieldNamesInTheSubTree(leftSubTree);
+                        fillFieldNamesInTheSubTree(leftSubTree, context);
                     }
                     if (checkRightSubTreeMetadata) {
-                        fillFieldNamesInTheSubTree(rightSubTree);
+                        fillFieldNamesInTheSubTree(rightSubTree, context);
                     }
 
                     // Applies the plan transformation using chosen index.
@@ -426,6 +447,10 @@
                         return false;
                     }
 
+                    if (checkApplicableOnly) {
+                        return true;
+                    }
+
                     // Finally, tries to apply plan transformation using the chosen index.
                     boolean res = chosenIndex.first.applyJoinPlanTransformation(afterJoinRefs, joinRef, leftSubTree,
                             rightSubTree, chosenIndex.second, analysisCtx, context, isLeftOuterJoin,
@@ -487,12 +512,13 @@
     }
 
     private boolean rewriteLocallyAndTransform(Mutable<ILogicalOperator> opRef, IOptimizationContext context,
-            IIntroduceAccessMethodRuleLocalRewrite<AbstractBinaryJoinOperator> rewriter) throws AlgebricksException {
+            IIntroduceAccessMethodRuleLocalRewrite<AbstractBinaryJoinOperator> rewriter, boolean checkApplicableOnly)
+            throws AlgebricksException {
         AbstractBinaryJoinOperator joinRewrite = rewriter.createOperator(joinOp, context);
         boolean transformationResult = false;
         if (joinRewrite != null) {
             Mutable<ILogicalOperator> joinRuleInput = new MutableObject<>(joinRewrite);
-            transformationResult = checkAndApplyJoinTransformation(joinRuleInput, context);
+            transformationResult = checkAndApplyJoinTransformation(joinRuleInput, context, checkApplicableOnly);
         }
 
         // Restore our state, so we can look for more optimizations if this transformation failed.
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 d9b5da9..69b7d56 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
@@ -54,6 +54,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.prettyprint.IPlanPrettyPrinter;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 
 /**
@@ -176,7 +177,44 @@
         afterSelectRefs = new ArrayList<>();
         // Recursively check the given plan whether the desired pattern exists in it.
         // If so, try to optimize the plan.
-        boolean planTransformed = checkAndApplyTheSelectTransformation(opRef, context);
+        List<Pair<IAccessMethod, Index>> chosenIndexes = new ArrayList<>();
+        Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = null;
+        boolean planTransformed =
+                checkAndApplyTheSelectTransformation(opRef, context, false, chosenIndexes, analyzedAMs);
+
+        if (selectOp != null) {
+            // We found an optimization here. Don't need to optimize this operator again.
+            context.addToDontApplySet(this, selectOp);
+        }
+
+        if (!planTransformed) {
+            return false;
+        } else {
+            OperatorPropertiesUtil.typeOpRec(opRef, context);
+        }
+
+        return planTransformed;
+    }
+
+    public boolean checkApplicable(Mutable<ILogicalOperator> opRef, IOptimizationContext context,
+            List<Pair<IAccessMethod, Index>> chosenIndexes, Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs)
+            throws AlgebricksException {
+        clear();
+        setMetadataDeclarations(context);
+
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+
+        // Already checked?
+        if (context.checkIfInDontApplySet(this, op)) {
+            return false;
+        }
+
+        afterSelectRefs = new ArrayList<>();
+        // Recursively check the given plan whether the desired pattern exists in it.
+        // If so, try to optimize the plan.
+
+        boolean planTransformed =
+                checkAndApplyTheSelectTransformation(opRef, context, true, chosenIndexes, analyzedAMs);
 
         if (selectOp != null) {
             // We found an optimization here. Don't need to optimize this operator again.
@@ -256,7 +294,7 @@
      *
      * @param chosenIndexes
      * @return Pair<IAccessMethod, Index> for the primary index
-     *         null otherwise
+     * null otherwise
      * @throws AlgebricksException
      */
     private Pair<IAccessMethod, Index> fetchPrimaryIndexAmongChosenIndexes(
@@ -328,11 +366,13 @@
      * if it is not already optimized.
      */
     protected boolean checkAndApplyTheSelectTransformation(Mutable<ILogicalOperator> opRef,
-            IOptimizationContext context) throws AlgebricksException {
+            IOptimizationContext context, boolean checkApplicableOnly, List<Pair<IAccessMethod, Index>> chosenIndexes,
+            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) throws AlgebricksException {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
         boolean selectFoundAndOptimizationApplied;
         boolean isSelectOp = false;
 
+        IPlanPrettyPrinter pp = context.getPrettyPrinter();
         Mutable<ILogicalOperator> selectRefFromThisOp = null;
         SelectOperator selectOpFromThisOp = null;
 
@@ -351,7 +391,8 @@
         // Recursively check the plan and try to optimize it. We first check the children of the given operator
         // to make sure an earlier select in the path is optimized first.
         for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
-            selectFoundAndOptimizationApplied = checkAndApplyTheSelectTransformation(inputOpRef, context);
+            selectFoundAndOptimizationApplied = checkAndApplyTheSelectTransformation(inputOpRef, context,
+                    checkApplicableOnly, chosenIndexes, analyzedAMs);
             if (selectFoundAndOptimizationApplied) {
                 return true;
             }
@@ -376,8 +417,7 @@
 
             // For each access method, contains the information about
             // whether an available index can be applicable or not.
-            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = null;
-            if (continueCheck) {
+            if (!checkApplicableOnly && continueCheck) {
                 analyzedAMs = new TreeMap<>();
             }
 
@@ -386,14 +426,14 @@
                 // If there exists a composite atomic-array index, our conjuncts will be split across multiple
                 // SELECTs. This rewrite is to be used **solely** for the purpose of changing a DATA-SCAN into a
                 // non-index-only plan branch. No nodes introduced from this rewrite will be used beyond this point.
-                if (rewriteLocallyAndTransform(selectRef, context, mergedSelectRewrite)) {
+                if (!checkApplicableOnly && rewriteLocallyAndTransform(selectRef, context, mergedSelectRewrite)) {
                     return true;
                 }
 
                 // If there exists a SUBPLAN in our plan, and we are conditioning on a variable, attempt to rewrite
                 // this subplan to allow an array-index AM to be introduced. Again, this rewrite is to be used
                 // **solely** for the purpose of changing a DATA-SCAN into a non-index-only plan branch.
-                if (rewriteLocallyAndTransform(selectRef, context, selectFromSubplanRewrite)) {
+                if (!checkApplicableOnly && rewriteLocallyAndTransform(selectRef, context, selectFromSubplanRewrite)) {
                     return true;
                 }
             }
@@ -426,20 +466,22 @@
                 fillSubTreeIndexExprs(subTree, analyzedAMs, context, false);
 
                 // Prune the access methods based on the function expression and access methods.
-                pruneIndexCandidates(analyzedAMs, context, typeEnvironment);
+                pruneIndexCandidates(analyzedAMs, context, typeEnvironment, false);
 
                 // Choose all indexes that will be applied.
-                List<Pair<IAccessMethod, Index>> chosenIndexes = chooseAllIndexes(analyzedAMs);
+                chooseAllIndexes(analyzedAMs, chosenIndexes);
 
                 if (chosenIndexes == null || chosenIndexes.isEmpty()) {
                     // We can't apply any index for this SELECT operator
                     context.addToDontApplySet(this, selectRef.getValue());
                     return false;
                 }
+                if (checkApplicableOnly) {
+                    return true;
+                }
 
                 // Apply plan transformation using chosen index.
                 boolean res;
-
                 // Primary index applicable?
                 Pair<IAccessMethod, Index> chosenPrimaryIndex = fetchPrimaryIndexAmongChosenIndexes(chosenIndexes);
                 if (chosenPrimaryIndex != null) {
@@ -453,7 +495,7 @@
                     AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenIndexes.get(0).first);
 
                     // Finds the field name of each variable in the sub-tree.
-                    fillFieldNamesInTheSubTree(subTree);
+                    fillFieldNamesInTheSubTree(subTree, context);
 
                     // Finally, try to apply plan transformation using chosen index.
                     res = chosenIndexes.get(0).first.applySelectPlanTransformation(afterSelectRefs, selectRef, subTree,
@@ -497,7 +539,10 @@
         boolean transformationResult = false;
         if (selectRewrite != null) {
             Mutable<ILogicalOperator> selectRuleInput = new MutableObject<>(selectRewrite);
-            transformationResult = checkAndApplyTheSelectTransformation(selectRuleInput, context);
+            List<Pair<IAccessMethod, Index>> chosenIndexes = new ArrayList<>();
+            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = null;
+            transformationResult =
+                    checkAndApplyTheSelectTransformation(selectRuleInput, context, false, chosenIndexes, analyzedAMs);
         }
 
         // Restore our state, so we can look for more optimizations if this transformation failed.
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 62812aa..031bc74 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
@@ -949,7 +949,8 @@
     }
 
     @Override
-    public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) throws AlgebricksException {
+    public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr, boolean checkApplicableOnly)
+            throws AlgebricksException {
         if (AccessMethodUtils.skipSecondaryIndexRequestedByAnnotation(index, optFuncExpr)) {
             return false;
         }
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 c55edf9..bbfb365 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
@@ -59,7 +59,7 @@
  */
 public class OptimizableOperatorSubTree {
 
-    public static enum DataSourceType {
+    public enum DataSourceType {
         DATASOURCE_SCAN,
         EXTERNAL_SCAN,
         PRIMARY_INDEX_LOOKUP,
@@ -68,6 +68,16 @@
         NO_DATASOURCE
     }
 
+    public static class RecordTypeSource {
+        final ARecordType recordType;
+        final int sourceIndicator;
+
+        RecordTypeSource(ARecordType recordType, int sourceIndicator) {
+            this.recordType = recordType;
+            this.sourceIndicator = sourceIndicator;
+        }
+    }
+
     private ILogicalOperator root = null;
     private Mutable<ILogicalOperator> rootRef = null;
     private final List<Mutable<ILogicalOperator>> assignsAndUnnestsRefs = new ArrayList<>();
@@ -91,6 +101,7 @@
     private List<DataSourceType> ixJoinOuterAdditionalDataSourceTypes = null;
     private List<Dataset> ixJoinOuterAdditionalDatasets = null;
     private List<ARecordType> ixJoinOuterAdditionalRecordTypes = null;
+    private final Map<LogicalVariable, RecordTypeSource> varsToRecordType = new HashMap<>();
 
     /**
      * Identifies the root of the subtree and initializes the data-source, assign, and unnest information.
@@ -172,10 +183,10 @@
                             AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
                             jobGenParams.readFromFuncArgs(f.getArguments());
                             if (jobGenParams.isPrimaryIndex()) {
-                                intializeDataSourceRefAndType(DataSourceType.PRIMARY_INDEX_LOOKUP, subTreeOpRef);
+                                initializeDataSourceRefAndType(DataSourceType.PRIMARY_INDEX_LOOKUP, subTreeOpRef);
                                 dataSourceFound = true;
                             } else if (unnestMapOp.getGenerateCallBackProceedResultVar()) {
-                                intializeDataSourceRefAndType(DataSourceType.INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP,
+                                initializeDataSourceRefAndType(DataSourceType.INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP,
                                         subTreeOpRef);
                                 dataSourceFound = true;
                             }
@@ -221,7 +232,7 @@
         return false;
     }
 
-    private void intializeDataSourceRefAndType(DataSourceType dsType, Mutable<ILogicalOperator> opRef) {
+    private void initializeDataSourceRefAndType(DataSourceType dsType, Mutable<ILogicalOperator> opRef) {
         if (getDataSourceRef() == null) {
             setDataSourceRef(opRef);
             setDataSourceType(dsType);
@@ -238,12 +249,6 @@
      * Also sets recordType to be the type of that dataset.
      */
     public boolean setDatasetAndTypeMetadata(MetadataProvider metadataProvider) throws AlgebricksException {
-        DataverseName dataverseName = null;
-        String datasetName = null;
-
-        Dataset ds = null;
-        ARecordType rType = null;
-
         List<Mutable<ILogicalOperator>> sourceOpRefs = new ArrayList<>();
         List<DataSourceType> dsTypes = new ArrayList<>();
 
@@ -259,6 +264,9 @@
         }
 
         for (int i = 0; i < sourceOpRefs.size(); i++) {
+            List<LogicalVariable> vars;
+            DataverseName dataverseName;
+            String datasetName;
             switch (dsTypes.get(i)) {
                 case DATASOURCE_SCAN:
                     DataSourceScanOperator dataSourceScan = (DataSourceScanOperator) sourceOpRefs.get(i).getValue();
@@ -272,6 +280,7 @@
                     Pair<DataverseName, String> datasetInfo = AnalysisUtil.getDatasetInfo(dataSourceScan);
                     dataverseName = datasetInfo.first;
                     datasetName = datasetInfo.second;
+                    vars = dataSourceScan.getScanVariables();
                     break;
                 case PRIMARY_INDEX_LOOKUP:
                 case INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP:
@@ -282,12 +291,14 @@
                     jobGenParams.readFromFuncArgs(f.getArguments());
                     datasetName = jobGenParams.getDatasetName();
                     dataverseName = jobGenParams.getDataverseName();
+                    vars = unnestMapOp.getScanVariables();
                     break;
                 case EXTERNAL_SCAN:
                     UnnestMapOperator externalScan = (UnnestMapOperator) sourceOpRefs.get(i).getValue();
                     datasetInfo = AnalysisUtil.getExternalDatasetInfo(externalScan);
                     dataverseName = datasetInfo.first;
                     datasetName = datasetInfo.second;
+                    vars = externalScan.getScanVariables();
                     break;
                 case COLLECTION_SCAN:
                     if (i != 0) {
@@ -303,7 +314,7 @@
                 return false;
             }
             // Find the dataset corresponding to the datasource in the metadata.
-            ds = metadataProvider.findDataset(dataverseName, datasetName);
+            Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
             if (ds == null) {
                 throw new CompilationException(ErrorCode.NO_METADATA_FOR_DATASET, root.getSourceLocation(),
                         datasetName);
@@ -318,26 +329,31 @@
                     getIxJoinOuterAdditionalRecordTypes().add(null);
                 }
             }
-            rType = (ARecordType) itemType;
+            ARecordType rType = (ARecordType) itemType;
 
             // Get the meta record type for that dataset.
-            IAType metaItemType =
-                    metadataProvider.findType(ds.getMetaItemTypeDataverseName(), ds.getMetaItemTypeName());
+            ARecordType metaItemType = (ARecordType) metadataProvider.findType(ds.getMetaItemTypeDataverseName(),
+                    ds.getMetaItemTypeName());
 
             // First index is always the primary datasource in this subtree.
             if (i == 0) {
                 setDataset(ds);
                 setRecordType(rType);
-                setMetaRecordType((ARecordType) metaItemType);
+                setMetaRecordType(metaItemType);
             } else {
                 getIxJoinOuterAdditionalDatasets().add(ds);
                 getIxJoinOuterAdditionalRecordTypes().add(rType);
             }
 
-            dataverseName = null;
-            datasetName = null;
-            ds = null;
-            rType = null;
+            if (!vars.isEmpty()) {
+                int numVars = vars.size();
+                if (ds.hasMetaPart()) {
+                    varsToRecordType.put(vars.get(numVars - 2), new RecordTypeSource(rType, 0));
+                    varsToRecordType.put(vars.get(numVars - 1), new RecordTypeSource(metaItemType, 1));
+                } else {
+                    varsToRecordType.put(vars.get(numVars - 1), new RecordTypeSource(rType, 0));
+                }
+            }
         }
 
         return true;
@@ -364,17 +380,6 @@
         return getDataSourceType() == DataSourceType.DATASOURCE_SCAN;
     }
 
-    public boolean hasIxJoinOuterAdditionalDataSourceScan() {
-        if (getIxJoinOuterAdditionalDataSourceTypes() != null) {
-            for (int i = 0; i < getIxJoinOuterAdditionalDataSourceTypes().size(); i++) {
-                if (getIxJoinOuterAdditionalDataSourceTypes().get(i) == DataSourceType.DATASOURCE_SCAN) {
-                    return true;
-                }
-            }
-        }
-        return false;
-    }
-
     public void reset() {
         setRoot(null);
         setRootRef(null);
@@ -392,6 +397,7 @@
         setIxJoinOuterAdditionalRecordTypes(null);
         lastMatchedDataSourceVars.first = -1;
         lastMatchedDataSourceVars.second = -1;
+        varsToRecordType.clear();
     }
 
     /**
@@ -545,6 +551,10 @@
         return recordType;
     }
 
+    public RecordTypeSource getRecordTypeFor(LogicalVariable var) {
+        return varsToRecordType.get(var);
+    }
+
     public void setRecordType(ARecordType recordType) {
         this.recordType = recordType;
     }
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 6959543..3de78f7 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
@@ -381,7 +381,7 @@
     }
 
     @Override
-    public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
+    public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr, boolean checkApplicableOnly) {
         if (AccessMethodUtils.skipSecondaryIndexRequestedByAnnotation(index, optFuncExpr)) {
             return false;
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java
index ba70aff..9a370c6 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java
@@ -156,9 +156,7 @@
             switch (workingOriginalOperator.getOperatorTag()) {
                 case UNNEST:
                     UnnestOperator originalUnnest = (UnnestOperator) workingOriginalOperator;
-                    UnnestOperator newUnnest =
-                            new UnnestOperator(originalUnnest.getVariable(), originalUnnest.getExpressionRef());
-                    newUnnest.setSourceLocation(sourceLocation);
+                    UnnestOperator newUnnest = (UnnestOperator) OperatorManipulationUtil.deepCopy(originalUnnest);
                     workingNewOperator.getInputs().add(new MutableObject<>(newUnnest));
                     workingNewOperator = newUnnest;
                     bottommostNewUnnest = (UnnestOperator) workingNewOperator;
@@ -166,8 +164,7 @@
 
                 case ASSIGN:
                     AssignOperator originalAssign = (AssignOperator) workingOriginalOperator;
-                    AssignOperator newAssign =
-                            new AssignOperator(originalAssign.getVariables(), originalAssign.getExpressions());
+                    AssignOperator newAssign = (AssignOperator) OperatorManipulationUtil.deepCopy(originalAssign);
                     newAssign.setSourceLocation(sourceLocation);
                     workingNewOperator.getInputs().add(new MutableObject<>(newAssign));
                     workingNewOperator = newAssign;
@@ -508,7 +505,7 @@
                 if (splitIntoConjuncts(conjunct.getValue(), innerExprConjuncts)) {
                     conjuncts.addAll(innerExprConjuncts);
                 } else {
-                    conjuncts.add(conjunct);
+                    conjuncts.add(new MutableObject<>(conjunct.getValue().cloneExpression()));
                 }
             }
             return true;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EnumerateJoinsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EnumerateJoinsRule.java
new file mode 100644
index 0000000..4a37007
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EnumerateJoinsRule.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.optimizer.rules.cbo;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.annotations.IndexedNLJoinExpressionAnnotation;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableInt;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.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.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.HashJoinExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+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.operators.logical.AbstractBinaryJoinOperator;
+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.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.prettyprint.IPlanPrettyPrinter;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class EnumerateJoinsRule implements IAlgebraicRewriteRule {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    protected final JoinEnum joinEnum;
+
+    public EnumerateJoinsRule(JoinEnum joinEnum) {
+        this.joinEnum = joinEnum;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+        return false;
+    }
+
+    /**
+     * If this method returns false, that means CBO code will not be used to optimize the part of the join graph that
+     * was passed in. Currently, we do not optimize query graphs with outer joins in them. If the CBO code is activated
+     * a new join graph (with inputs possibly switched) will be created and the return value will be true.
+     */
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        boolean cboMode = this.getCBOMode(context);
+        boolean cboTestMode = this.getCBOTestMode(context);
+        if (!(cboMode || cboTestMode)) {
+            return false;
+        }
+        // If we reach here, then either cboMode or cboTestMode is true.
+        // If cboTestMode is true, then we use predefined cardinalities for datasets for asterixdb regression tests.
+        // If cboMode is true, then all datasets need to have samples, otherwise the check in doAllDataSourcesHaveSamples()
+        // further below will return false.
+        ILogicalOperator op = opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.INNERJOIN) {
+            return false;
+        }
+
+        // if this join has already been seen before, no need to apply the rule again
+        if (context.checkIfInDontApplySet(this, op)) {
+            return false;
+        }
+
+        List<ILogicalOperator> joinOps = new ArrayList<>();
+        List<ILogicalOperator> internalEdges = new ArrayList<>();
+        HashMap<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap = new HashMap<>();
+        // The data scan operators. Will be in the order of the from clause.
+        // Important for position ordering when assigning bits to join expressions.
+        List<Pair<EmptyTupleSourceOperator, DataSourceScanOperator>> emptyTupleAndDataSourceOps = new ArrayList<>();
+        HashMap<DataSourceScanOperator, EmptyTupleSourceOperator> dataSourceEmptyTupleHashMap = new HashMap<>();
+
+        IPlanPrettyPrinter pp = context.getPrettyPrinter();
+        printPlan(pp, (AbstractLogicalOperator) op, "Original Whole plan1");
+        boolean canTransform = getJoinOpsAndLeafInputs(op, emptyTupleAndDataSourceOps, joinLeafInputsHashMap,
+                dataSourceEmptyTupleHashMap, internalEdges, joinOps);
+
+        if (!canTransform) {
+            return false;
+        }
+
+        // if this happens, something in the input plan is not acceptable to the new code.
+        if (emptyTupleAndDataSourceOps.size() != joinLeafInputsHashMap.size()) {
+            throw new IllegalStateException(
+                    "ETS " + emptyTupleAndDataSourceOps.size() + " != LI " + joinLeafInputsHashMap.size());
+        }
+
+        printPlan(pp, (AbstractLogicalOperator) op, "Original Whole plan2");
+
+        int numberOfFromTerms = emptyTupleAndDataSourceOps.size();
+
+        joinEnum.initEnum((AbstractLogicalOperator) op, cboMode, cboTestMode, numberOfFromTerms,
+                emptyTupleAndDataSourceOps, joinLeafInputsHashMap, dataSourceEmptyTupleHashMap, internalEdges, joinOps,
+                context);
+
+        if (cboMode) {
+            if (!doAllDataSourcesHaveSamples(emptyTupleAndDataSourceOps, context)) {
+                return false;
+            }
+        }
+
+        printPlan(pp, (AbstractLogicalOperator) op, "Before calling new code. same plan still??");
+        int cheapestPlan = joinEnum.enumerateJoins();
+        printPlan(pp, (AbstractLogicalOperator) op, "After join enumeration. Must return same plan??");
+        if (cheapestPlan == PlanNode.NO_PLAN) {
+            return false;
+        }
+
+        PlanNode cheapestPlanNode = joinEnum.allPlans.get(cheapestPlan);
+        checkForMultipleUsesOfVariablesInJoinPreds(cheapestPlanNode, joinLeafInputsHashMap, context);
+        buildNewTree(cheapestPlanNode, joinLeafInputsHashMap, joinOps, new MutableInt(0));
+        ILogicalOperator root = addConstantInternalEdgesAtTheTop(joinOps.get(0), internalEdges);
+
+        printPlan(pp, (AbstractLogicalOperator) joinOps.get(0), "New Whole Plan after buildNewTree");
+        printPlan(pp, (AbstractLogicalOperator) root, "New Whole Plan after buildNewTree");
+
+        // this will be the new root
+        opRef.setValue(root);
+
+        if (LOGGER.isTraceEnabled()) {
+            LOGGER.trace("---------------------------- Printing Leaf Inputs");
+            printLeafPlans(pp, joinLeafInputsHashMap);
+            // print joins starting from the bottom
+            for (int i = joinOps.size() - 1; i >= 0; i--) {
+                printPlan(pp, (AbstractLogicalOperator) joinOps.get(i), "join " + i);
+            }
+            printPlan(pp, (AbstractLogicalOperator) joinOps.get(0), "New Whole Plan");
+            printPlan(pp, (AbstractLogicalOperator) root, "New Whole Plan");
+        }
+
+        // turn of this rule for all joins in this set (subtree)
+        for (ILogicalOperator joinOp : joinOps) {
+            context.addToDontApplySet(this, joinOp);
+        }
+        return true;
+    }
+
+    private boolean getCBOMode(IOptimizationContext context) {
+        PhysicalOptimizationConfig physOptConfig = context.getPhysicalOptimizationConfig();
+        return physOptConfig.getCBOMode();
+    }
+
+    private boolean getCBOTestMode(IOptimizationContext context) {
+        PhysicalOptimizationConfig physOptConfig = context.getPhysicalOptimizationConfig();
+        return physOptConfig.getCBOTestMode();
+    }
+
+    /**
+     * Should not see any kind of joins here. store the emptyTupeSourceOp and DataSource operators.
+     * Each leaf input will normally have both, but sometimes only emptyTupeSourceOp will be present (in lists)
+     */
+    private Pair<EmptyTupleSourceOperator, DataSourceScanOperator> containsLeafInputOnly(ILogicalOperator op) {
+        DataSourceScanOperator dataSourceOp = null;
+        ILogicalOperator currentOp = op;
+        while (currentOp.getInputs().size() == 1) {
+            if (currentOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+                // we should not see two data scans in the same path
+                if (dataSourceOp != null) {
+                    return null;
+                }
+                dataSourceOp = (DataSourceScanOperator) currentOp;
+            }
+            currentOp = currentOp.getInputs().get(0).getValue();
+        }
+        if (currentOp.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE) {
+            return new Pair<>((EmptyTupleSourceOperator) currentOp, dataSourceOp);
+        }
+        return null;
+    }
+
+    /**
+     * Check to see if there is only one assign here and nothing below that other than a join.
+     * have not seen cases where there is more than one assign in a leafinput.
+    */
+    private boolean onlyOneAssign(ILogicalOperator nextOp) {
+        if (nextOp.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+            return false;
+        }
+        List<Mutable<ILogicalOperator>> nextOpInputs = nextOp.getInputs();
+        return nextOpInputs.get(0).getValue().getOperatorTag() == LogicalOperatorTag.INNERJOIN;
+    }
+
+    /**
+     * This is the main routines that stores all the join operators and the leafInputs. We will later reuse the same
+     * join operators but switch the leafInputs (see buildNewTree). The whole scheme is based on the assumption that the
+     * leafInputs can be switched. The various data structures make the leafInputs accessible efficiently.
+     */
+    private boolean getJoinOpsAndLeafInputs(ILogicalOperator op,
+            List<Pair<EmptyTupleSourceOperator, DataSourceScanOperator>> emptyTupleAndDataSourceOps,
+            HashMap<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap,
+            HashMap<DataSourceScanOperator, EmptyTupleSourceOperator> dataSourceEmptyTupleHashMap,
+            List<ILogicalOperator> internalEdges, List<ILogicalOperator> joinOps) {
+        if (op.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN) {
+            return false;
+        }
+        for (Mutable<ILogicalOperator> nextOp : op.getInputs()) {
+            boolean canTransform = getJoinOpsAndLeafInputs(nextOp.getValue(), emptyTupleAndDataSourceOps,
+                    joinLeafInputsHashMap, dataSourceEmptyTupleHashMap, internalEdges, joinOps);
+            if (!canTransform) {
+                return false;
+            }
+        }
+        if (op.getOperatorTag() == LogicalOperatorTag.INNERJOIN) {
+            joinOps.add(op);
+            // follow the inputs and see if they reach a datascan operator
+            for (int i = 0; i < 2; i++) {
+                ILogicalOperator nextOp = op.getInputs().get(i).getValue();
+                Pair<EmptyTupleSourceOperator, DataSourceScanOperator> etsDataSource = containsLeafInputOnly(nextOp);
+                if (etsDataSource == null) {
+                    // this means that we did not find a emptyTupleSourceOp operator. Could be an internal edge
+                    if (nextOp.getOperatorTag() != LogicalOperatorTag.INNERJOIN) {
+                        if (onlyOneAssign(nextOp)) {
+                            // currently, will handle only assign statement and nothing else in an internal Edge.
+                            // we can lift this restriction later if the need arises. This just makes some code easier.
+                            internalEdges.add(nextOp);
+                        } else {
+                            return false;
+                        }
+                    }
+                } else {
+                    EmptyTupleSourceOperator etsOp = etsDataSource.first;
+                    DataSourceScanOperator dataSourceOp = etsDataSource.second;
+                    emptyTupleAndDataSourceOps.add(new Pair<>(etsOp, dataSourceOp));
+                    joinLeafInputsHashMap.put(etsOp, nextOp);
+                    dataSourceEmptyTupleHashMap.put(dataSourceOp, etsOp);
+                }
+            }
+        }
+        return true;
+    }
+
+    private void addCardCostAnnotations(ILogicalOperator op, PlanNode plan) {
+        op.getAnnotations().put(OperatorAnnotations.OP_OUTPUT_CARDINALITY,
+                (double) Math.round(plan.getJoinNode().getCardinality() * 100) / 100);
+        op.getAnnotations().put(OperatorAnnotations.OP_COST_TOTAL,
+                (double) Math.round(plan.computeTotalCost() * 100) / 100);
+        if (plan.IsScanNode()) {
+            op.getAnnotations().put(OperatorAnnotations.OP_INPUT_CARDINALITY,
+                    (double) Math.round(plan.getJoinNode().getOrigCardinality() * 100) / 100);
+            op.getAnnotations().put(OperatorAnnotations.OP_COST_LOCAL,
+                    (double) Math.round(plan.computeOpCost() * 100) / 100);
+        } else {
+            op.getAnnotations().put(OperatorAnnotations.OP_LEFT_EXCHANGE_COST,
+                    (double) Math.round(plan.getLeftExchangeCost() * 100) / 100);
+            op.getAnnotations().put(OperatorAnnotations.OP_RIGHT_EXCHANGE_COST,
+                    (double) Math.round(plan.getRightExchangeCost() * 100) / 100);
+            op.getAnnotations().put(OperatorAnnotations.OP_COST_LOCAL,
+                    (double) Math.round(
+                            (plan.computeOpCost() - plan.getLeftExchangeCost() - plan.getRightExchangeCost()) * 100)
+                            / 100);
+        }
+
+        if (op.getOperatorTag().equals(LogicalOperatorTag.SELECT)) {
+            op.getAnnotations().put(OperatorAnnotations.OP_COST_LOCAL, 0.0);
+        }
+    }
+
+    /**
+     * Finds the DataSourceScanOperator given a leafInput
+     */
+    private ILogicalOperator findDataSourceScanOperator(ILogicalOperator op) {
+        ILogicalOperator origOp = op;
+        while (op != null && op.getOperatorTag() != LogicalOperatorTag.EMPTYTUPLESOURCE) {
+            if (op.getOperatorTag().equals(LogicalOperatorTag.DATASOURCESCAN)) {
+                return op;
+            }
+            op = op.getInputs().get(0).getValue();
+        }
+        return origOp;
+    }
+
+    private void removeJoinAnnotations(AbstractFunctionCallExpression afcExpr) {
+        afcExpr.removeAnnotation(BroadcastExpressionAnnotation.class);
+        afcExpr.removeAnnotation(IndexedNLJoinExpressionAnnotation.class);
+        afcExpr.removeAnnotation(HashJoinExpressionAnnotation.class);
+    }
+
+    private void setAnnotation(AbstractFunctionCallExpression afcExpr, IExpressionAnnotation anno) {
+        FunctionIdentifier fi = afcExpr.getFunctionIdentifier();
+        List<Mutable<ILogicalExpression>> arguments = afcExpr.getArguments();
+        int argumentCount = arguments.size();
+
+        if (fi.equals(AlgebricksBuiltinFunctions.AND)) {
+            for (int i = 0; i < argumentCount; i++) {
+                ILogicalExpression argument = arguments.get(i).getValue();
+                AbstractFunctionCallExpression expr = (AbstractFunctionCallExpression) argument;
+                expr.putAnnotation(anno);
+            }
+        } else {
+            afcExpr.putAnnotation(anno);
+        }
+    }
+
+    private void buildNewTree(PlanNode plan, HashMap<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap,
+            List<ILogicalOperator> joinOps, MutableInt totalNumberOfJoins) {
+        // we have to move the inputs in op around so that they match the tree structure in pn
+        // we use the existing joinOps and switch the leafInputs appropriately.
+        List<PlanNode> allPlans = joinEnum.getAllPlans();
+        int leftIndex = plan.getLeftPlanIndex();
+        int rightIndex = plan.getRightPlanIndex();
+        PlanNode leftPlan = allPlans.get(leftIndex);
+        PlanNode rightPlan = allPlans.get(rightIndex);
+        ILogicalOperator joinOp = joinOps.get(totalNumberOfJoins.intValue());
+
+        if (plan.IsJoinNode()) {
+            AbstractBinaryJoinOperator abJoinOp = (AbstractBinaryJoinOperator) joinOp;
+            ILogicalExpression expr = plan.getJoinExpr();
+            abJoinOp.getCondition().setValue(expr);
+            // add the annotations
+            if (plan.getJoinOp() == PlanNode.JoinMethod.INDEX_NESTED_LOOP_JOIN) {
+                // this annotation is needed for the physical optimizer to replace this with the unnest operator later
+                AbstractFunctionCallExpression afcExpr = (AbstractFunctionCallExpression) expr;
+                removeJoinAnnotations(afcExpr);
+                setAnnotation(afcExpr, IndexedNLJoinExpressionAnnotation.INSTANCE_ANY_INDEX);
+            } else if (plan.getJoinOp() == PlanNode.JoinMethod.HYBRID_HASH_JOIN
+                    || plan.getJoinOp() == PlanNode.JoinMethod.BROADCAST_HASH_JOIN
+                    || plan.getJoinOp() == PlanNode.JoinMethod.CARTESIAN_PRODUCT_JOIN) {
+                if (plan.getJoinOp() == PlanNode.JoinMethod.BROADCAST_HASH_JOIN) {
+                    // Broadcast the right branch.
+                    BroadcastExpressionAnnotation bcast =
+                            new BroadcastExpressionAnnotation(plan.side == HashJoinExpressionAnnotation.BuildSide.RIGHT
+                                    ? BroadcastExpressionAnnotation.BroadcastSide.RIGHT
+                                    : BroadcastExpressionAnnotation.BroadcastSide.LEFT);
+                    AbstractFunctionCallExpression afcExpr = (AbstractFunctionCallExpression) expr;
+                    removeJoinAnnotations(afcExpr);
+                    setAnnotation(afcExpr, bcast);
+                } else if (plan.getJoinOp() == PlanNode.JoinMethod.HYBRID_HASH_JOIN) {
+                    HashJoinExpressionAnnotation hjAnnotation = new HashJoinExpressionAnnotation(plan.side);
+                    AbstractFunctionCallExpression afcExpr = (AbstractFunctionCallExpression) expr;
+                    removeJoinAnnotations(afcExpr);
+                    setAnnotation(afcExpr, hjAnnotation);
+                } else {
+                    if (expr != ConstantExpression.TRUE) {
+                        AbstractFunctionCallExpression afcExpr = (AbstractFunctionCallExpression) expr;
+                        removeJoinAnnotations(afcExpr);
+                    }
+                }
+            }
+            addCardCostAnnotations(joinOp, plan);
+        }
+
+        if (leftPlan.IsScanNode()) {
+            // leaf
+            ILogicalOperator leftInput = joinLeafInputsHashMap.get(leftPlan.getEmptyTupleSourceOp());
+            if (leftInput.getOperatorTag() == LogicalOperatorTag.SELECT) {
+                addCardCostAnnotations(leftInput, leftPlan);
+            }
+            joinOp.getInputs().get(0).setValue(leftInput);
+            addCardCostAnnotations(findDataSourceScanOperator(leftInput), leftPlan);
+        } else {
+            // join
+            totalNumberOfJoins.increment();
+            ILogicalOperator leftInput = joinOps.get(totalNumberOfJoins.intValue());
+            joinOp.getInputs().get(0).setValue(leftInput);
+            buildNewTree(allPlans.get(leftIndex), joinLeafInputsHashMap, joinOps, totalNumberOfJoins);
+        }
+
+        if (rightPlan.IsScanNode()) {
+            // leaf
+            ILogicalOperator rightInput = joinLeafInputsHashMap.get(rightPlan.getEmptyTupleSourceOp());
+            if (rightInput.getOperatorTag() == LogicalOperatorTag.SELECT) {
+                addCardCostAnnotations(rightInput, rightPlan);
+            }
+            joinOp.getInputs().get(1).setValue(rightInput);
+            addCardCostAnnotations(findDataSourceScanOperator(rightInput), rightPlan);
+        } else {
+            // join
+            totalNumberOfJoins.increment();
+            ILogicalOperator rightInput = joinOps.get(totalNumberOfJoins.intValue());
+            joinOp.getInputs().get(1).setValue(rightInput);
+            buildNewTree(allPlans.get(rightIndex), joinLeafInputsHashMap, joinOps, totalNumberOfJoins);
+        }
+    }
+
+    // in some very rare cases, there is an internal edge that has an assign statement such as $$var = 20 but this variable
+    // is not used anywhere in the current join graph but is used outside the current join graph. So we add this assign to the top of
+    // our plan, so the rest of the code will be happy. Strange that this assign appears in the join graph.
+    private ILogicalOperator addConstantInternalEdgesAtTheTop(ILogicalOperator op,
+            List<ILogicalOperator> internalEdges) {
+        ILogicalOperator root = op;
+        for (ILogicalOperator ie : internalEdges) {
+            // this will be an Assign, so no need to check
+            AssignOperator aOp = (AssignOperator) ie;
+            aOp.getInputs().get(0).setValue(root);
+            root = aOp;
+        }
+        return root;
+    }
+
+    public static void printPlan(IPlanPrettyPrinter pp, AbstractLogicalOperator op, String text)
+            throws AlgebricksException {
+        if (LOGGER.isTraceEnabled()) {
+            pp.reset();
+            pp.printOperator(op, true);
+            LOGGER.trace("---------------------------- {}\n{}\n----------------------------", text, pp);
+        }
+    }
+
+    private void printLeafPlans(IPlanPrettyPrinter pp,
+            HashMap<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap) throws AlgebricksException {
+        Iterator<Map.Entry<EmptyTupleSourceOperator, ILogicalOperator>> li =
+                joinLeafInputsHashMap.entrySet().iterator();
+        int i = 0;
+        while (li.hasNext()) {
+            Map.Entry<EmptyTupleSourceOperator, ILogicalOperator> pair = li.next();
+            ILogicalOperator element = pair.getValue();
+            printPlan(pp, (AbstractLogicalOperator) element, "Printing Leaf Input" + i);
+            i++;
+        }
+    }
+
+    private boolean allEqualityPreds(ILogicalExpression expr) {
+        List<Mutable<ILogicalExpression>> conjs = new ArrayList<>();
+        // check that the expr is AND(EQ(), EQ(),..)
+        if (expr.splitIntoConjuncts(conjs)) {
+            for (Mutable<ILogicalExpression> conj : conjs) {
+                if (!(((AbstractFunctionCallExpression) conj.getValue()).getFunctionIdentifier()
+                        .equals(AlgebricksBuiltinFunctions.EQ))) {
+                    return false;
+                }
+            }
+            return true;
+        }
+        return false;
+    }
+
+    // This routine should not be needed! When the same variable is used multiple times in a join predicate as in
+    // AND(eq($$25, $$27), eq($$25, $$34)), JoinUtils.isHashJoinCondition() returns false which makes no sense.
+    // Tried changing the above routine but it always lead to some failures. Unable to figure out what the problem was,
+    // we just replace the duplicate occurrence of every variable with a new variable following by appropriate assign.
+    // We do this only once just once before we construct the final plan.
+    private void checkForMultipleUsesOfVariablesInJoinPreds(PlanNode plan,
+            HashMap<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap, IOptimizationContext context)
+            throws AlgebricksException {
+        List<PlanNode> allPlans = joinEnum.getAllPlans();
+        if (plan.IsJoinNode()) {
+            ILogicalExpression exp = plan.getJoinExpr();
+            if (!allEqualityPreds(exp)) {
+                return;
+            }
+            boolean changes = true;
+            while (changes) {
+                changes = false;
+                List<LogicalVariable> vars = new ArrayList<>();
+                exp.getUsedVariables(vars);
+                Set<LogicalVariable> set = new LinkedHashSet<>();
+                set.addAll(vars);
+                if (set.size() < vars.size()) {
+                    // walk thru vars and find the first instance of the duplicate
+                    for (int i = 0; i < vars.size() - 1; i++) {
+                        for (int j = i + 1; j < vars.size(); j++) {
+                            if (vars.get(i) == vars.get(j)) {
+                                /// find the leafInout that contains this vars(i)
+                                for (Map.Entry<EmptyTupleSourceOperator, ILogicalOperator> mapElement : joinLeafInputsHashMap
+                                        .entrySet()) {
+                                    ILogicalOperator joinLeafInput = mapElement.getValue();
+                                    EmptyTupleSourceOperator ets = mapElement.getKey();
+                                    HashSet<LogicalVariable> vars2 = new HashSet<>();
+                                    VariableUtilities.getLiveVariables(joinLeafInput, vars2);
+                                    if (vars2.contains(vars.get(i))) {
+                                        LogicalVariable newVar = context.newVar();
+                                        // replace one occurrence of vars(i) in exp
+                                        substituteVarOnce(exp, vars.get(i), newVar);
+                                        VariableReferenceExpression oldvarExpr =
+                                                new VariableReferenceExpression(vars.get(i));
+                                        AssignOperator assign =
+                                                new AssignOperator(newVar, new MutableObject<>(oldvarExpr));
+                                        // Now add an assign to the joinLeafInput : newvar <-- oldvar
+                                        assign.getInputs().add(new MutableObject<>(joinLeafInput));
+                                        context.computeAndSetTypeEnvironmentForOperator(assign);
+                                        context.addNotToBeInlinedVar(newVar);
+                                        context.addNotToBeInlinedVar(vars.get(i));
+
+                                        // also update the joinLeafInputsHashMap
+                                        joinLeafInputsHashMap.put(ets, assign);
+                                        changes = true;
+                                    }
+                                }
+                            }
+                        }
+                    }
+                }
+            }
+            // now traverse left and right side plans
+            int leftIndex = plan.getLeftPlanIndex();
+            int rightIndex = plan.getRightPlanIndex();
+            PlanNode leftPlan = allPlans.get(leftIndex);
+            PlanNode rightPlan = allPlans.get(rightIndex);
+            if (leftPlan.IsJoinNode()) {
+                checkForMultipleUsesOfVariablesInJoinPreds(leftPlan, joinLeafInputsHashMap, context);
+            }
+            if (rightPlan.IsJoinNode()) {
+                checkForMultipleUsesOfVariablesInJoinPreds(rightPlan, joinLeafInputsHashMap, context);
+            }
+        }
+    }
+
+    private boolean substituteVarOnce(ILogicalExpression exp, LogicalVariable oldVar, LogicalVariable newVar) {
+        switch (exp.getExpressionTag()) {
+            case FUNCTION_CALL:
+                AbstractFunctionCallExpression fun = (AbstractFunctionCallExpression) exp;
+                for (int i = 0; i < fun.getArguments().size(); i++) {
+                    ILogicalExpression arg = fun.getArguments().get(i).getValue();
+                    if (substituteVarOnce(arg, oldVar, newVar)) {
+                        return true;
+                    }
+                }
+                return false;
+            case VARIABLE:
+                VariableReferenceExpression varExpr = (VariableReferenceExpression) exp;
+                if (varExpr.getVariableReference().equals(oldVar)) {
+                    varExpr.setVariable(newVar);
+                    return true;
+                }
+                return false;
+            default:
+                return false;
+        }
+    }
+
+    // check to see if every dataset has a sample. If not, CBO code cannot run. A warning message must be issued as well.
+    private boolean doAllDataSourcesHaveSamples(
+            List<Pair<EmptyTupleSourceOperator, DataSourceScanOperator>> emptyTupleAndDataSourceOps,
+            IOptimizationContext context) throws AlgebricksException {
+        for (int i = 0; i < emptyTupleAndDataSourceOps.size(); i++) {
+            if (emptyTupleAndDataSourceOps.get(i).getSecond() != null) {
+                DataSourceScanOperator scanOp = emptyTupleAndDataSourceOps.get(i).getSecond();
+                Index index = joinEnum.getStatsHandle().findSampleIndex(scanOp, context);
+                if (index == null) {
+                    return false;
+                }
+            }
+        }
+        return true;
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EstimatedCostComputationVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EstimatedCostComputationVisitor.java
new file mode 100644
index 0000000..9b9fd28
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/EstimatedCostComputationVisitor.java
@@ -0,0 +1,352 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.cbo;
+
+import java.util.Map;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
+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.ForwardOperator;
+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;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteUpsertOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SplitOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+
+/**
+ * A visitor that annotates an operator with its estimated cardinality and estimated cost.
+ */
+public class EstimatedCostComputationVisitor implements ILogicalOperatorVisitor<Pair<Double, Double>, Double> {
+
+    public EstimatedCostComputationVisitor() {
+    }
+
+    @Override
+    public Pair<Double, Double> visitAggregateOperator(AggregateOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitRunningAggregateOperator(RunningAggregateOperator op, Double arg)
+            throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Double arg)
+            throws AlgebricksException {
+        // Empty tuple source operator sends an empty tuple to downstream operators.
+        return new Pair<>(1.0, 1.0);
+    }
+
+    @Override
+    public Pair<Double, Double> visitGroupByOperator(GroupByOperator op, Double arg) throws AlgebricksException {
+        // Needs more work in the cardinality estimation code to estimate group by cardinality and cost.
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitLimitOperator(LimitOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitInnerJoinOperator(InnerJoinOperator op, Double arg) throws AlgebricksException {
+        return visitInnerJoin(op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Double arg)
+            throws AlgebricksException {
+        return visitLeftOuterUnnest(op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Double arg)
+            throws AlgebricksException {
+        Pair<Double, Double> cardCost = annotate(this, op, arg);
+        return op.getDataSourceReference().getValue().getOperatorTag() == LogicalOperatorTag.SUBPLAN ? cardCost
+                : new Pair<>(0.0, 0.0);
+    }
+
+    @Override
+    public Pair<Double, Double> visitOrderOperator(OrderOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitAssignOperator(AssignOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitWindowOperator(WindowOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitSelectOperator(SelectOperator op, Double arg) throws AlgebricksException {
+        Pair<Double, Double> cardCost = op.getInputs().get(0).getValue().accept(this, arg);
+        for (Map.Entry<String, Object> anno : op.getAnnotations().entrySet()) {
+            if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_OUTPUT_CARDINALITY)) {
+                cardCost.setFirst((Double) anno.getValue());
+            } else if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_COST_TOTAL)) {
+                cardCost.setSecond((Double) anno.getValue());
+            }
+        }
+
+        return cardCost;
+    }
+
+    @Override
+    public Pair<Double, Double> visitDelegateOperator(DelegateOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitProjectOperator(ProjectOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitReplicateOperator(ReplicateOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitSplitOperator(SplitOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitMaterializeOperator(MaterializeOperator op, Double arg)
+            throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitScriptOperator(ScriptOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitSubplanOperator(SubplanOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitSinkOperator(SinkOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitUnionOperator(UnionAllOperator op, Double arg) throws AlgebricksException {
+        // Needs more work.
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitUnnestOperator(UnnestOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitLeftOuterUnnestOperator(LeftOuterUnnestOperator op, Double arg)
+            throws AlgebricksException {
+        return visitLeftOuterUnnest(op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitUnnestMapOperator(UnnestMapOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Double arg)
+            throws AlgebricksException {
+        return visitLeftOuterUnnest(op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitDataScanOperator(DataSourceScanOperator op, Double arg)
+            throws AlgebricksException {
+        Pair<Double, Double> cardCost = new Pair<>(1.0, 1.0);
+
+        for (Map.Entry<String, Object> anno : op.getAnnotations().entrySet()) {
+            if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_INPUT_CARDINALITY)) {
+                cardCost.setFirst((Double) anno.getValue());
+            } else if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_COST_TOTAL)) {
+                cardCost.setSecond((Double) anno.getValue());
+            }
+        }
+
+        return cardCost;
+    }
+
+    @Override
+    public Pair<Double, Double> visitDistinctOperator(DistinctOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitExchangeOperator(ExchangeOperator op, Double arg) throws AlgebricksException {
+        double exchCost = 0;
+        if (arg != null) {
+            exchCost = arg;
+        }
+
+        Pair<Double, Double> cardCost = op.getInputs().get(0).getValue().accept(this, arg);
+        if (exchCost != 0) {
+            op.getAnnotations().put(OperatorAnnotations.OP_COST_LOCAL, exchCost);
+            op.getAnnotations().put(OperatorAnnotations.OP_COST_TOTAL, exchCost + cardCost.getSecond());
+        } else {
+            op.getAnnotations().put(OperatorAnnotations.OP_COST_LOCAL, cardCost.getSecond());
+            op.getAnnotations().put(OperatorAnnotations.OP_COST_TOTAL, cardCost.getSecond());
+        }
+        op.getAnnotations().put(OperatorAnnotations.OP_OUTPUT_CARDINALITY, cardCost.getFirst());
+        return cardCost;
+    }
+
+    @Override
+    public Pair<Double, Double> visitWriteOperator(WriteOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitDistributeResultOperator(DistributeResultOperator op, Double arg)
+            throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitWriteResultOperator(WriteResultOperator op, Double arg)
+            throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitInsertDeleteUpsertOperator(InsertDeleteUpsertOperator op, Double arg)
+            throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, Double arg)
+            throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitTokenizeOperator(TokenizeOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitForwardOperator(ForwardOperator op, Double arg) throws AlgebricksException {
+        return annotate(this, op, arg);
+    }
+
+    @Override
+    public Pair<Double, Double> visitIntersectOperator(IntersectOperator op, Double arg) throws AlgebricksException {
+        // Needs more work.
+        return annotate(this, op, arg);
+    }
+
+    private static Pair<Double, Double> annotate(EstimatedCostComputationVisitor visitor, ILogicalOperator op,
+            Double arg) throws AlgebricksException {
+        if (op.getInputs().isEmpty()) {
+            return new Pair<>(1.0, 1.0);
+        }
+        Pair<Double, Double> cardCost = op.getInputs().get(0).getValue().accept(visitor, arg);
+        op.getAnnotations().put(OperatorAnnotations.OP_OUTPUT_CARDINALITY, cardCost.getFirst());
+        op.getAnnotations().put(OperatorAnnotations.OP_COST_TOTAL, cardCost.getSecond());
+        op.getAnnotations().put(OperatorAnnotations.OP_COST_LOCAL, cardCost.getSecond());
+        return cardCost;
+    }
+
+    // Visits an operator that has the left outer semantics, e.g.,
+    // left outer join, left outer unnest, left outer unnest map.
+    private Pair<Double, Double> visitLeftOuterUnnest(ILogicalOperator operator, Double arg)
+            throws AlgebricksException {
+        // Needs more work.
+        return operator.getInputs().get(0).getValue().accept(this, arg);
+    }
+
+    // Visits an inner join operator, particularly, deals with the case the join is a cartesian product.
+    private Pair<Double, Double> visitInnerJoin(InnerJoinOperator joinOperator, Double arg) throws AlgebricksException {
+        Pair<Double, Double> cardCost = new Pair<>(1.0, 1.0);
+
+        ILogicalOperator left = joinOperator.getInputs().get(0).getValue();
+        ILogicalOperator right = joinOperator.getInputs().get(1).getValue();
+        double leftExchangeCost = 0;
+        double rightExchangeCost = 0;
+
+        for (Map.Entry<String, Object> anno : joinOperator.getAnnotations().entrySet()) {
+            if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_OUTPUT_CARDINALITY)) {
+                cardCost.setFirst((Double) anno.getValue());
+            } else if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_COST_TOTAL)) {
+                cardCost.setSecond((Double) anno.getValue());
+            } else if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_LEFT_EXCHANGE_COST)) {
+                leftExchangeCost = (double) anno.getValue();
+            } else if (anno.getValue() != null && anno.getKey().equals(OperatorAnnotations.OP_RIGHT_EXCHANGE_COST)) {
+                rightExchangeCost = (double) anno.getValue();
+            }
+        }
+
+        // Visit the left and right branches.
+        left.accept(this, leftExchangeCost);
+        right.accept(this, rightExchangeCost);
+
+        return cardCost;
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinCondition.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinCondition.java
new file mode 100644
index 0000000..bc7ea3f
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinCondition.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.optimizer.rules.cbo;
+
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class JoinCondition {
+
+    public static final int NO_JC = -1;
+
+    public ILogicalExpression joinCondition;
+    public boolean derived = false;
+    public boolean partOfComposite = false;
+    public int numberOfVars = 0; // how many variables
+    public int componentNumber = 0; // for identifying if join graph is connected
+    public int datasetBits;
+    // used for triangle detection; we strictly do not mean left and right here.
+    // first and second sides would be more appropriate
+    public int leftSideBits;
+    public int rightSideBits;
+    public double selectivity;
+    public comparisonOp comparisonType;
+
+    public enum comparisonOp {
+        OP_EQ,
+        OP_OTHER
+    }
+
+    public ILogicalExpression getJoinCondition() {
+        return joinCondition;
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinEnum.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinEnum.java
new file mode 100644
index 0000000..b1a6fb6
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinEnum.java
@@ -0,0 +1,779 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.cbo;
+
+import java.time.LocalDateTime;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.annotations.IndexedNLJoinExpressionAnnotation;
+import org.apache.asterix.metadata.declared.DataSource;
+import org.apache.asterix.metadata.declared.DataSourceId;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.om.base.AOrderedList;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.optimizer.cost.Cost;
+import org.apache.asterix.optimizer.cost.CostMethods;
+import org.apache.asterix.optimizer.cost.ICost;
+import org.apache.asterix.optimizer.cost.ICostMethods;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.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.expressions.BroadcastExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.HashJoinExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+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.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.prettyprint.IPlanPrettyPrinter;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class JoinEnum {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    protected List<JoinCondition> joinConditions; // "global" list of join conditions
+    protected List<PlanNode> allPlans; // list of all plans
+    protected JoinNode[] jnArray; // array of all join nodes
+    protected int jnArraySize;
+    protected List<Pair<EmptyTupleSourceOperator, DataSourceScanOperator>> emptyTupleAndDataSourceOps;
+    protected Map<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap;
+    protected Map<DataSourceScanOperator, EmptyTupleSourceOperator> dataSourceEmptyTupleHashMap;
+    protected List<ILogicalExpression> singleDatasetPreds;
+    protected List<ILogicalOperator> internalEdges;
+    protected List<ILogicalOperator> joinOps;
+    protected ILogicalOperator localJoinOp; // used in nestedLoopsApplicable code.
+    protected IOptimizationContext optCtx;
+    protected Stats stats;
+    protected PhysicalOptimizationConfig physOptConfig;
+    protected boolean cboMode;
+    protected boolean cboTestMode;
+    protected int numberOfTerms;
+    protected AbstractLogicalOperator op;
+    protected boolean connectedJoinGraph;
+    protected boolean forceJoinOrderMode;
+    protected String queryPlanShape;
+    protected ICost cost;
+    protected ICostMethods costMethods;
+
+    public JoinEnum() {
+    }
+
+    public void initEnum(AbstractLogicalOperator op, boolean cboMode, boolean cboTestMode, int numberOfFromTerms,
+            List<Pair<EmptyTupleSourceOperator, DataSourceScanOperator>> emptyTupleAndDataSourceOps,
+            Map<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap,
+            Map<DataSourceScanOperator, EmptyTupleSourceOperator> dataSourceEmptyTupleHashMap,
+            List<ILogicalOperator> internalEdges, List<ILogicalOperator> joinOps, IOptimizationContext context) {
+        this.singleDatasetPreds = new ArrayList<>();
+        this.joinConditions = new ArrayList<>();
+        this.internalEdges = new ArrayList<>();
+        this.allPlans = new ArrayList<>();
+        this.numberOfTerms = numberOfFromTerms;
+        this.cboMode = cboMode;
+        this.cboTestMode = cboTestMode;
+        this.connectedJoinGraph = true;
+        this.optCtx = context;
+        this.physOptConfig = context.getPhysicalOptimizationConfig();
+        this.emptyTupleAndDataSourceOps = emptyTupleAndDataSourceOps;
+        this.joinLeafInputsHashMap = joinLeafInputsHashMap;
+        this.dataSourceEmptyTupleHashMap = dataSourceEmptyTupleHashMap;
+        this.internalEdges = internalEdges;
+        this.joinOps = joinOps;
+        this.op = op;
+        this.forceJoinOrderMode = getForceJoinOrderMode(context);
+        this.queryPlanShape = getQueryPlanShape(context);
+        initCostHandleAndJoinNodes(context);
+    }
+
+    protected void initCostHandleAndJoinNodes(IOptimizationContext context) {
+        this.cost = new Cost();
+        this.costMethods = new CostMethods(context);
+        this.stats = new Stats(optCtx, this);
+        this.jnArraySize = (int) Math.pow(2.0, this.numberOfTerms);
+        this.jnArray = new JoinNode[this.jnArraySize];
+        // initialize all the join nodes
+        for (int i = 0; i < this.jnArraySize; i++) {
+            this.jnArray[i] = new JoinNode(i, this);
+        }
+    }
+
+    public List<JoinCondition> getJoinConditions() {
+        return joinConditions;
+    }
+
+    public List<PlanNode> getAllPlans() {
+        return allPlans;
+    }
+
+    public JoinNode[] getJnArray() {
+        return jnArray;
+    }
+
+    public Cost getCostHandle() {
+        return (Cost) cost;
+    }
+
+    public CostMethods getCostMethodsHandle() {
+        return (CostMethods) costMethods;
+    }
+
+    public Stats getStatsHandle() {
+        return stats;
+    }
+
+    public Map<EmptyTupleSourceOperator, ILogicalOperator> getJoinLeafInputsHashMap() {
+        return joinLeafInputsHashMap;
+    }
+
+    public Map<DataSourceScanOperator, EmptyTupleSourceOperator> getDataSourceEmptyTupleHashMap() {
+        return dataSourceEmptyTupleHashMap;
+    }
+
+    public ILogicalOperator findLeafInput(List<LogicalVariable> logicalVars) throws AlgebricksException {
+        Set<LogicalVariable> vars = new HashSet<>();
+        for (int pos = 0; pos < emptyTupleAndDataSourceOps.size(); pos++) {
+            EmptyTupleSourceOperator emptyOp = emptyTupleAndDataSourceOps.get(pos).getFirst();
+            ILogicalOperator op = joinLeafInputsHashMap.get(emptyOp);
+            vars.clear();
+            // this is expensive to do. So store this once and reuse
+            VariableUtilities.getLiveVariables(op, vars);
+            if (vars.containsAll(logicalVars)) {
+                return op;
+            }
+        }
+        // this will never happen, but keep compiler happy
+        return null;
+    }
+
+    public ILogicalExpression combineAllConditions(List<Integer> newJoinConditions) {
+        if (newJoinConditions.size() == 0) {
+            // this is a cartesian product
+            return ConstantExpression.TRUE;
+        }
+        if (newJoinConditions.size() == 1) {
+            JoinCondition jc = joinConditions.get(newJoinConditions.get(0));
+            return jc.joinCondition;
+        }
+        ScalarFunctionCallExpression andExpr = new ScalarFunctionCallExpression(
+                BuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.AND));
+
+        for (int joinNum : newJoinConditions) {
+            // Need to AND all the expressions.
+            JoinCondition jc = joinConditions.get(joinNum);
+            andExpr.getArguments().add(new MutableObject<>(jc.joinCondition));
+        }
+        return andExpr;
+    }
+
+    public ILogicalExpression getNestedLoopJoinExpr(List<Integer> newJoinConditions) {
+        if (newJoinConditions.size() != 1) {
+            // may remove this restriction later if possible
+            return null;
+        }
+        JoinCondition jc = joinConditions.get(newJoinConditions.get(0));
+        return jc.joinCondition;
+    }
+
+    public ILogicalExpression getHashJoinExpr(List<Integer> newJoinConditions) {
+        if (newJoinConditions.size() == 0) {
+            // this is a cartesian product
+            return ConstantExpression.TRUE;
+        }
+        if (newJoinConditions.size() == 1) {
+            JoinCondition jc = joinConditions.get(newJoinConditions.get(0));
+            if (jc.comparisonType == JoinCondition.comparisonOp.OP_EQ) {
+                return jc.joinCondition;
+            }
+            return null;
+        }
+        ScalarFunctionCallExpression andExpr = new ScalarFunctionCallExpression(
+                BuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.AND));
+
+        // at least one equality predicate needs to be present for a hash join to be possible.
+        boolean eqPredFound = false;
+        for (int joinNum : newJoinConditions) {
+            // need to AND all the expressions.
+            JoinCondition jc = joinConditions.get(joinNum);
+            if (jc.comparisonType == JoinCondition.comparisonOp.OP_EQ) {
+                eqPredFound = true;
+            }
+            andExpr.getArguments().add(new MutableObject<>(jc.joinCondition));
+        }
+        // return null if no equality predicates were found
+        return eqPredFound ? andExpr : null;
+    }
+
+    public HashJoinExpressionAnnotation.BuildSide findHashJoinHint(List<Integer> newJoinConditions) {
+        for (int i : newJoinConditions) {
+            JoinCondition jc = joinConditions.get(i);
+            if (jc.comparisonType != JoinCondition.comparisonOp.OP_EQ) {
+                return null;
+            }
+            ILogicalExpression expr = jc.joinCondition;
+            if (expr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+                AbstractFunctionCallExpression AFCexpr = (AbstractFunctionCallExpression) expr;
+                HashJoinExpressionAnnotation hjea = AFCexpr.getAnnotation(HashJoinExpressionAnnotation.class);
+                if (hjea != null) {
+                    return hjea.getBuildSide();
+                }
+            }
+        }
+        return null;
+    }
+
+    public BroadcastExpressionAnnotation.BroadcastSide findBroadcastHashJoinHint(List<Integer> newJoinConditions) {
+        for (int i : newJoinConditions) {
+            JoinCondition jc = joinConditions.get(i);
+            if (jc.comparisonType != JoinCondition.comparisonOp.OP_EQ) {
+                return null;
+            }
+            ILogicalExpression expr = jc.joinCondition;
+            if (expr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+                AbstractFunctionCallExpression AFCexpr = (AbstractFunctionCallExpression) expr;
+                BroadcastExpressionAnnotation bcasthjea = AFCexpr.getAnnotation(BroadcastExpressionAnnotation.class);
+                if (bcasthjea != null) {
+                    return bcasthjea.getBroadcastSide();
+                }
+            }
+        }
+        return null;
+    }
+
+    public boolean findNLJoinHint(List<Integer> newJoinConditions) {
+        for (int i : newJoinConditions) {
+            JoinCondition jc = joinConditions.get(i);
+            ILogicalExpression expr = jc.joinCondition;
+            if (expr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+                AbstractFunctionCallExpression AFCexpr = (AbstractFunctionCallExpression) expr;
+                IndexedNLJoinExpressionAnnotation inljea =
+                        AFCexpr.getAnnotation(IndexedNLJoinExpressionAnnotation.class);
+                if (inljea != null) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+    public int findJoinNodeIndexByName(String name) {
+        for (int i = 1; i <= this.numberOfTerms; i++) {
+            if (name.equals(jnArray[i].datasetNames.get(0))) {
+                return i;
+            } else if (name.equals(jnArray[i].alias)) {
+                return i;
+            }
+        }
+        // should never happen; keep compiler happy.
+        return JoinNode.NO_JN;
+    }
+
+    public int findJoinNodeIndex(LogicalVariable lv) throws AlgebricksException {
+        List<Pair<EmptyTupleSourceOperator, DataSourceScanOperator>> emptyTupleAndDataSourceOps =
+                this.emptyTupleAndDataSourceOps;
+        Map<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap = this.joinLeafInputsHashMap;
+
+        for (Map.Entry<EmptyTupleSourceOperator, ILogicalOperator> mapElement : joinLeafInputsHashMap.entrySet()) {
+            ILogicalOperator joinLeafInput = mapElement.getValue();
+            HashSet<LogicalVariable> vars = new HashSet<>();
+            // this should get the variables from the inputs only, since the join condition is itself set to null
+            VariableUtilities.getLiveVariables(joinLeafInput, vars);
+            if (vars.contains(lv)) {
+                EmptyTupleSourceOperator key = mapElement.getKey();
+                for (int i = 0; i < emptyTupleAndDataSourceOps.size(); i++) {
+                    if (key.equals(emptyTupleAndDataSourceOps.get(i).getFirst())) {
+                        return i;
+                    }
+                }
+            }
+        }
+        return JoinNode.NO_JN;
+    }
+
+    private int findBits(LogicalVariable lv) throws AlgebricksException {
+        int idx = findJoinNodeIndex(lv);
+        if (idx >= 0) {
+            return 1 << idx;
+        }
+
+        // so this variable must be in an internal edge in an assign statement. Find the RHS variables there
+        for (ILogicalOperator op : this.internalEdges) {
+            if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                List<LogicalVariable> vars2 = new ArrayList<>();
+                VariableUtilities.getUsedVariables(op, vars2);
+                int bits = 0;
+                for (LogicalVariable lv2 : vars2) {
+                    bits |= findBits(lv2);
+                }
+                return bits;
+            }
+        }
+        // should never reach this because every variable must exist in some leaf input.
+        return JoinNode.NO_JN;
+    }
+
+    // This finds all the join Conditions in the whole query. This is a global list of all join predicates.
+    // It also fills in the dataset Bits for each join predicate.
+    protected void findJoinConditions() throws AlgebricksException {
+        List<Mutable<ILogicalExpression>> conjs = new ArrayList<>();
+        for (ILogicalOperator jOp : joinOps) {
+            AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) jOp;
+            ILogicalExpression expr = joinOp.getCondition().getValue();
+            conjs.clear();
+            if (expr.splitIntoConjuncts(conjs)) {
+                conjs.remove(new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
+                for (Mutable<ILogicalExpression> conj : conjs) {
+                    JoinCondition jc = new JoinCondition();
+                    jc.joinCondition = conj.getValue().cloneExpression();
+                    joinConditions.add(jc);
+                    jc.selectivity = stats.getSelectivityFromAnnotationMain(jc.joinCondition, true);
+                }
+            } else {
+                if ((expr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL))) {
+                    JoinCondition jc = new JoinCondition();
+                    // change to not a true condition
+                    jc.joinCondition = expr.cloneExpression();
+                    joinConditions.add(jc);
+                    jc.selectivity = stats.getSelectivityFromAnnotationMain(jc.joinCondition, true);
+                }
+            }
+        }
+
+        // now patch up any join conditions that have variables referenced in any internal assign statements.
+        List<LogicalVariable> usedVars = new ArrayList<>();
+        for (JoinCondition jc : joinConditions) {
+            usedVars.clear();
+            ILogicalExpression expr = jc.joinCondition;
+            expr.getUsedVariables(usedVars);
+            for (ILogicalOperator ie : internalEdges) {
+                AssignOperator aOp = (AssignOperator) ie;
+                for (int i = 0; i < aOp.getVariables().size(); i++) {
+                    if (usedVars.contains(aOp.getVariables().get(i))) {
+                        OperatorManipulationUtil.replaceVarWithExpr((AbstractFunctionCallExpression) expr,
+                                aOp.getVariables().get(i), aOp.getExpressions().get(i).getValue());
+                        jc.joinCondition = expr;
+                        jc.selectivity = stats.getSelectivityFromAnnotationMain(jc.joinCondition, true);
+                    }
+                }
+            }
+        }
+
+        // now fill the datasetBits for each join condition.
+        for (JoinCondition jc : joinConditions) {
+            ILogicalExpression joinExpr = jc.joinCondition;
+            /*
+            if (joinExpr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+                AbstractFunctionCallExpression afce = (AbstractFunctionCallExpression) joinExpr;
+                // remove all the join method type annotations.
+                afce.removeAnnotation(BroadcastExpressionAnnotation.class);
+                afce.removeAnnotation(IndexedNLJoinExpressionAnnotation.class);
+                afce.removeAnnotation(HashJoinExpressionAnnotation.class);
+            }
+             */
+            usedVars.clear();
+            joinExpr.getUsedVariables(usedVars);
+            // We only set these for join predicates that have exactly two tables
+            jc.leftSideBits = jc.rightSideBits = JoinCondition.NO_JC;
+            if (((AbstractFunctionCallExpression) joinExpr).getFunctionIdentifier()
+                    .equals(AlgebricksBuiltinFunctions.EQ)) {
+                jc.comparisonType = JoinCondition.comparisonOp.OP_EQ;
+            } else {
+                jc.comparisonType = JoinCondition.comparisonOp.OP_OTHER;
+            }
+            jc.numberOfVars = usedVars.size();
+
+            for (int i = 0; i < jc.numberOfVars; i++) {
+                int bits = findBits(usedVars.get(i)); // rename to findInWhichLeaf
+                if (bits != JoinCondition.NO_JC) {
+                    if (i == 0) {
+                        jc.leftSideBits = bits;
+                    } else if (i == 1) {
+                        jc.rightSideBits = bits;
+                    } else {
+                        // have to deal with preds such as r.a + s.a = 5 OR r.a + s.a = t.a
+                    }
+                    jc.datasetBits |= bits;
+                }
+            }
+        }
+    }
+
+    // in case we have l.partkey = ps.partkey and l.suppkey = ps.suppkey, we will only use the first one for cardinality computations.
+    // treat it like a Pk-Fk join; simplifies cardinality computation
+    private void markCompositeJoinPredicates() {
+        // can use dataSetBits??? This will be simpler.
+        for (int i = 0; i < joinConditions.size() - 1; i++) {
+            for (int j = i + 1; j < joinConditions.size(); j++) {
+                if (joinConditions.get(i).datasetBits == joinConditions.get(j).datasetBits) {
+                    joinConditions.get(j).partOfComposite = true;
+                }
+            }
+        }
+    }
+
+    private boolean verticesMatch(JoinCondition jc1, JoinCondition jc2) {
+        return jc1.leftSideBits == jc2.leftSideBits || jc1.leftSideBits == jc2.rightSideBits
+                || jc1.rightSideBits == jc2.leftSideBits || jc1.rightSideBits == jc2.rightSideBits;
+    }
+
+    private void markComponents(int startingJoinCondition) {
+        List<JoinCondition> joinConditions = this.getJoinConditions();
+        // see if all the joinCondition can be reached starting with the first.
+        JoinCondition jc1 = joinConditions.get(startingJoinCondition);
+        for (int i = 0; i < joinConditions.size(); i++) {
+            JoinCondition jc2 = joinConditions.get(i);
+            if (i != startingJoinCondition && jc2.componentNumber == 0) {
+                // a new edge not visited before
+                if (verticesMatch(jc1, jc2)) {
+                    jc2.componentNumber = 1;
+                    markComponents(i);
+                }
+            }
+        }
+    }
+
+    private void findIfJoinGraphIsConnected() {
+        int numJoinConditions = joinConditions.size();
+        if (numJoinConditions < numberOfTerms - 1) {
+            /// not enough join predicates
+            connectedJoinGraph = false;
+            return;
+        }
+        if (numJoinConditions > 0) {
+            joinConditions.get(0).componentNumber = 1;
+            markComponents(0);
+            for (int i = 1; i < numJoinConditions; i++) {
+                if (joinConditions.get(i).componentNumber == 0) {
+                    connectedJoinGraph = false;
+                    return;
+                }
+            }
+        }
+    }
+
+    private double findInListCard(ILogicalOperator op) {
+        if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+            return 1.0;
+        }
+
+        if (op.getOperatorTag() == LogicalOperatorTag.UNNEST) {
+            UnnestOperator unnestOp = (UnnestOperator) op;
+            ILogicalExpression unnestExpr = unnestOp.getExpressionRef().getValue();
+            UnnestingFunctionCallExpression unnestingFuncExpr = (UnnestingFunctionCallExpression) unnestExpr;
+
+            if (unnestingFuncExpr.getFunctionIdentifier().equals(BuiltinFunctions.SCAN_COLLECTION)) {
+                if (unnestingFuncExpr.getArguments().get(0).getValue()
+                        .getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+                    ConstantExpression constantExpr =
+                            (ConstantExpression) unnestingFuncExpr.getArguments().get(0).getValue();
+                    AsterixConstantValue constantValue = (AsterixConstantValue) constantExpr.getValue();
+                    IAObject v = constantValue.getObject();
+                    if (v.getType().getTypeTag() == ATypeTag.ARRAY) {
+                        AOrderedList array = (AOrderedList) v;
+                        return array.size();
+                    }
+                }
+            }
+        }
+        // just a guess
+        return 10.0;
+    }
+
+    private String findAlias(DataSourceScanOperator scanOp) {
+        DataSource ds = (DataSource) scanOp.getDataSource();
+        List<LogicalVariable> allVars = scanOp.getVariables();
+        LogicalVariable dataRecVarInScan = ds.getDataRecordVariable(allVars);
+        return dataRecVarInScan.toString().substring(2);
+    }
+
+    private int addNonBushyJoinNodes(int level, int jnNumber, int[] startJnAtLevel) throws AlgebricksException {
+        // adding joinNodes of level (2, 3, ..., numberOfTerms)
+        int startJnSecondLevel = startJnAtLevel[2];
+        int startJnPrevLevel = startJnAtLevel[level - 1];
+        int startJnNextLevel = startJnAtLevel[level];
+        int i, j, addPlansToThisJn;
+
+        // walking thru the previous level
+        for (i = startJnPrevLevel; i < startJnNextLevel; i++) {
+            JoinNode jnI = jnArray[i];
+            jnI.jnArrayIndex = i;
+            if (jnI.highestDatasetId == 0) {
+                // this jn can be skipped
+                continue;
+            }
+
+            // walk thru the first level here
+            for (j = 1; j < startJnSecondLevel; j++) {
+                if (level == 2 && i > j) {
+                    // don't want to generate x y and y x. we will do this in plan generation.
+                    continue;
+                }
+                JoinNode jnJ = jnArray[j];
+                jnJ.jnArrayIndex = j;
+                if ((jnI.datasetBits & jnJ.datasetBits) > 0) {
+                    // these already have some common table
+                    continue;
+                }
+                int newBits = jnI.datasetBits | jnJ.datasetBits;
+                JoinNode jnNewBits = jnArray[newBits];
+                jnNewBits.jnArrayIndex = newBits;
+                // visiting this join node for the first time
+                if (jnNewBits.jnIndex == 0) {
+                    jnNumber++;
+                    JoinNode jn = jnArray[jnNumber];
+                    jn.jnArrayIndex = jnNumber;
+                    // if we want to locate the joinNode num (say 33) which has tables 1, 2, and 5.
+                    // if these bits are turned on, we get 19. Then jn[19].jn_index will equal 33.
+                    // Then jn[33].highestKeyspaceId will equal 5
+                    // if this joinNode ever gets removed, then set jn[19].highestKeyspaceId = 0
+                    jn.datasetBits = newBits;
+                    jnNewBits.jnIndex = addPlansToThisJn = jnNumber;
+                    jn.level = level;
+                    jn.highestDatasetId = Math.max(jnI.highestDatasetId, j);
+
+                    jn.datasetIndexes = new ArrayList<>();
+                    jn.datasetIndexes.addAll(jnI.datasetIndexes);
+                    jn.datasetIndexes.addAll(jnJ.datasetIndexes);
+                    Collections.sort(jn.datasetIndexes);
+
+                    jn.datasetNames = new ArrayList<>();
+                    jn.datasetNames.addAll(jnI.datasetNames);
+                    jn.datasetNames.addAll(jnJ.datasetNames);
+                    Collections.sort(jn.datasetNames);
+                    jn.size = jnI.size + jnJ.size;
+                    jn.cardinality = jn.computeJoinCardinality();
+
+                } else {
+                    addPlansToThisJn = jnNewBits.jnIndex;
+                }
+
+                JoinNode jnIJ = jnArray[addPlansToThisJn];
+                jnIJ.jnArrayIndex = addPlansToThisJn;
+                jnIJ.addMultiDatasetPlans(jnI, jnJ, level);
+                if (forceJoinOrderMode) {
+                    break;
+                }
+            }
+            if (forceJoinOrderMode) {
+                break;
+            }
+        }
+
+        return jnNumber;
+    }
+
+    private int enumerateHigherLevelJoinNodes() throws AlgebricksException {
+        int jnNumber = this.numberOfTerms;
+        int[] firstJnAtLevel;
+        firstJnAtLevel = new int[numberOfTerms + 1];
+        firstJnAtLevel[1] = 1;
+        IPlanPrettyPrinter pp = optCtx.getPrettyPrinter();
+        // after implementing greedy plan, we can start at level 3;
+        int startLevel = 2;
+        if (LOGGER.isTraceEnabled()) {
+            EnumerateJoinsRule.printPlan(pp, op, "Original Whole plan in JN 4");
+        }
+        for (int level = startLevel; level <= numberOfTerms; level++) {
+            firstJnAtLevel[level] = jnNumber + 1;
+            jnNumber = addNonBushyJoinNodes(level, jnNumber, firstJnAtLevel);
+        }
+        if (LOGGER.isTraceEnabled()) {
+            EnumerateJoinsRule.printPlan(pp, op, "Original Whole plan in JN 5");
+        }
+        return jnNumber;
+    }
+
+    protected int enumerateBaseLevelJoinNodes() throws AlgebricksException {
+        int lastBaseLevelJnNum = initializeBaseLevelJoinNodes();
+        if (lastBaseLevelJnNum == PlanNode.NO_PLAN) {
+            return PlanNode.NO_PLAN;
+        }
+        int dataScanPlan = PlanNode.NO_PLAN;
+        for (int i = 1; i <= numberOfTerms; i++) {
+            JoinNode jn = jnArray[i];
+            EmptyTupleSourceOperator ets = emptyTupleAndDataSourceOps.get(i - 1).getFirst();
+            ILogicalOperator leafInput = joinLeafInputsHashMap.get(ets);
+            dataScanPlan = jn.addSingleDatasetPlans();
+            if (dataScanPlan == PlanNode.NO_PLAN) {
+                return PlanNode.NO_PLAN;
+            }
+            // We may not add any index plans, so need to check for NO_PLAN
+            jn.addIndexAccessPlans(leafInput);
+        }
+        return numberOfTerms;
+    }
+
+    protected int initializeBaseLevelJoinNodes() throws AlgebricksException {
+        // join nodes have been allocated in the JoinEnum
+        // add a dummy Plan Node; we do not want planNode at position 0 to be a valid plan
+        PlanNode pn = new PlanNode(0, this);
+        pn.jn = null;
+        pn.jnIndexes[0] = pn.jnIndexes[1] = JoinNode.NO_JN;
+        pn.planIndexes[0] = pn.planIndexes[1] = PlanNode.NO_PLAN;
+        pn.opCost = pn.totalCost = new Cost(0);
+        allPlans.add(pn);
+
+        boolean noCards = false;
+        // initialize the level 1 join nodes
+        for (int i = 1; i <= numberOfTerms; i++) {
+            JoinNode jn = jnArray[i];
+            jn.jnArrayIndex = i;
+            jn.datasetBits = 1 << (i - 1);
+            jn.datasetIndexes = new ArrayList<>(Collections.singleton(i));
+            EmptyTupleSourceOperator ets = emptyTupleAndDataSourceOps.get(i - 1).getFirst();
+            ILogicalOperator leafInput = joinLeafInputsHashMap.get(ets);
+
+            DataSourceScanOperator scanOp = emptyTupleAndDataSourceOps.get(i - 1).getSecond();
+            if (scanOp != null) {
+                DataSourceId id = (DataSourceId) scanOp.getDataSource().getId();
+                jn.alias = findAlias(scanOp);
+                jn.datasetNames = new ArrayList<>(Collections.singleton(id.getDatasourceName()));
+                Index.SampleIndexDetails idxDetails;
+                Index index = stats.findSampleIndex(scanOp, optCtx);
+                if (index != null) {
+                    idxDetails = (Index.SampleIndexDetails) index.getIndexDetails();
+                } else {
+                    idxDetails = null;
+                }
+
+                jn.idxDetails = idxDetails;
+                if (cboTestMode) {
+                    // to make asterix tests run
+                    jn.origCardinality = 1000000;
+                    jn.size = 500;
+                } else {
+                    if (idxDetails == null) {
+                        return PlanNode.NO_PLAN;
+                    }
+                    jn.setOrigCardinality(idxDetails.getSourceCardinality());
+                    jn.setAvgDocSize(idxDetails.getSourceAvgItemSize());
+                }
+                // multiply by the respective predicate selectivities
+                jn.cardinality = jn.origCardinality * stats.getSelectivity(leafInput, false);
+            } else {
+                // could be unnest or assign
+                jn.datasetNames = new ArrayList<>(Collections.singleton("unnestOrAssign"));
+                jn.origCardinality = jn.cardinality = findInListCard(leafInput);
+                // just a guess
+                jn.size = 10;
+            }
+
+            if (jn.origCardinality >= Cost.MAX_CARD) {
+                noCards = true;
+            }
+            jn.correspondingEmptyTupleSourceOp = emptyTupleAndDataSourceOps.get(i - 1).getFirst();
+            jn.highestDatasetId = i;
+            jn.level = 1;
+        }
+        if (noCards) {
+            return PlanNode.NO_PLAN;
+        }
+        return numberOfTerms;
+    }
+
+    // main entry point in this file
+    public int enumerateJoins() throws AlgebricksException {
+        // create a localJoinOp for use in calling existing nested loops code.
+        InnerJoinOperator dummyInput = new InnerJoinOperator(null, null, null);
+        localJoinOp = new InnerJoinOperator(new MutableObject<>(ConstantExpression.TRUE),
+                new MutableObject<>(dummyInput), new MutableObject<>(dummyInput));
+
+        int lastBaseLevelJnNum = enumerateBaseLevelJoinNodes();
+        if (lastBaseLevelJnNum == PlanNode.NO_PLAN) {
+            return PlanNode.NO_PLAN;
+        }
+
+        IPlanPrettyPrinter pp = optCtx.getPrettyPrinter();
+        if (LOGGER.isTraceEnabled()) {
+            EnumerateJoinsRule.printPlan(pp, op, "Original Whole plan in JN 1");
+        }
+
+        findJoinConditions();
+        findIfJoinGraphIsConnected();
+
+        if (LOGGER.isTraceEnabled()) {
+            EnumerateJoinsRule.printPlan(pp, op, "Original Whole plan in JN 2");
+        }
+
+        markCompositeJoinPredicates();
+        int lastJnNum = enumerateHigherLevelJoinNodes();
+        JoinNode lastJn = jnArray[lastJnNum];
+        if (LOGGER.isTraceEnabled()) {
+            EnumerateJoinsRule.printPlan(pp, op, "Original Whole plan in JN END");
+            LOGGER.trace(dumpJoinNodes(lastJnNum));
+        }
+
+        // find the cheapest plan
+        int cheapestPlanIndex = lastJn.cheapestPlanIndex;
+        if (LOGGER.isTraceEnabled() && cheapestPlanIndex > 0) {
+            LOGGER.trace("Cheapest Plan is {} number of terms is {} joinNodes {}", cheapestPlanIndex, numberOfTerms,
+                    lastJnNum);
+        }
+
+        return cheapestPlanIndex;
+    }
+
+    private String dumpJoinNodes(int numJoinNodes) {
+        StringBuilder sb = new StringBuilder(128);
+        sb.append(LocalDateTime.now());
+        for (int i = 1; i <= numJoinNodes; i++) {
+            JoinNode jn = jnArray[i];
+            sb.append(jn);
+        }
+        sb.append('\n').append("Printing cost of all Final Plans").append('\n');
+        jnArray[numJoinNodes].printCostOfAllPlans(sb);
+        return sb.toString();
+    }
+
+    public static boolean getForceJoinOrderMode(IOptimizationContext context) {
+        PhysicalOptimizationConfig physOptConfig = context.getPhysicalOptimizationConfig();
+        return physOptConfig.getForceJoinOrderMode();
+    }
+
+    public static String getQueryPlanShape(IOptimizationContext context) {
+        PhysicalOptimizationConfig physOptConfig = context.getPhysicalOptimizationConfig();
+        return physOptConfig.getQueryPlanShapeMode();
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinNode.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinNode.java
new file mode 100644
index 0000000..cf4fcad
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/JoinNode.java
@@ -0,0 +1,847 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.cbo;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+
+import org.apache.asterix.common.annotations.SkipSecondaryIndexSearchExpressionAnnotation;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.optimizer.cost.Cost;
+import org.apache.asterix.optimizer.cost.ICost;
+import org.apache.asterix.optimizer.rules.am.AccessMethodAnalysisContext;
+import org.apache.asterix.optimizer.rules.am.IAccessMethod;
+import org.apache.asterix.optimizer.rules.am.IOptimizableFuncExpr;
+import org.apache.asterix.optimizer.rules.am.IntroduceJoinAccessMethodRule;
+import org.apache.asterix.optimizer.rules.am.IntroduceSelectAccessMethodRule;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+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.BroadcastExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.HashJoinExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.PredicateCardinalityAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.config.AlgebricksConfig;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class JoinNode {
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    protected JoinEnum joinEnum;
+    protected int jnArrayIndex;
+    protected int datasetBits; // this is bitmap of all the keyspaceBits present in this joinNode
+    protected List<Integer> datasetIndexes;
+    protected List<String> datasetNames;
+    protected String alias;
+    protected int cheapestPlanIndex;
+    protected ICost cheapestPlanCost;
+    protected double origCardinality; // without any selections
+    protected double cardinality;
+    protected double size;
+    protected List<Integer> planIndexesArray; // indexes into the PlanNode array in enumerateJoins
+    protected int jnIndex, level, highestDatasetId;
+    protected JoinNode rightJn, leftJn;
+    protected List<Integer> applicableJoinConditions;
+    protected EmptyTupleSourceOperator correspondingEmptyTupleSourceOp; // There is a 1-1 relationship between the LVs and the dataSourceScanOps and the leafInputs.
+    protected List<Pair<IAccessMethod, Index>> chosenIndexes;
+    protected Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs;
+    protected Index.SampleIndexDetails idxDetails;
+    protected static int NO_JN = -1;
+    protected static int NO_CARDS = -2;
+
+    public JoinNode(int i) {
+        this.jnArrayIndex = i;
+        planIndexesArray = new ArrayList<>();
+        cheapestPlanIndex = PlanNode.NO_PLAN;
+        size = 1; // for now, will be the size of the doc for this joinNode
+    }
+
+    public JoinNode(int i, JoinEnum joinE) {
+        this(i);
+        joinEnum = joinE;
+        cheapestPlanCost = joinEnum.getCostHandle().maxCost();
+    }
+
+    public boolean IsBaseLevelJoinNode() {
+        if (this.jnArrayIndex <= joinEnum.numberOfTerms) {
+            return true;
+        }
+        return false;
+    }
+
+    public boolean IsHigherLevelJoinNode() {
+        return !IsBaseLevelJoinNode();
+    }
+
+    public double computeJoinCardinality() {
+        JoinNode[] jnArray = joinEnum.getJnArray();
+        List<JoinCondition> joinConditions = joinEnum.getJoinConditions();
+
+        this.applicableJoinConditions = new ArrayList<>();
+        findApplicableJoinConditions();
+
+        if (LOGGER.isTraceEnabled() && this.applicableJoinConditions.size() == 0) {
+            LOGGER.trace("applicable Join Conditions size is 0 in join Node " + this.jnArrayIndex);
+        }
+
+        // Wonder if this computation will result in an overflow exception. Better to multiply them with selectivities also.
+        double productJoinCardinality = 1.0;
+        for (int idx : this.datasetIndexes) {
+            productJoinCardinality *= jnArray[idx].cardinality;
+        }
+
+        double productJoinSels = 1.0;
+        for (int idx : this.applicableJoinConditions) {
+            if (!joinConditions.get(idx).partOfComposite) {
+                productJoinSels *= joinConditions.get(idx).selectivity;
+            }
+        }
+        return productJoinCardinality * productJoinSels;
+    }
+
+    public double getCardinality() {
+        return cardinality;
+    }
+
+    public void setCardinality(double card) {
+        cardinality = card;
+    }
+
+    public double getOrigCardinality() {
+        return origCardinality;
+    }
+
+    public void setOrigCardinality(double card) {
+        origCardinality = card;
+    }
+
+    public void setAvgDocSize(double avgDocSize) {
+        size = avgDocSize;
+    }
+
+    public double getInputSize() {
+        return size;
+    }
+
+    public double getOutputSize() {
+        return size; // need to change this to account for projections
+    }
+
+    public JoinNode getLeftJn() {
+        return leftJn;
+    }
+
+    public JoinNode getRightJn() {
+        return rightJn;
+    }
+
+    public String getAlias() {
+        return alias;
+    }
+
+    public List<String> getDatasetNames() {
+        return datasetNames;
+    }
+
+    public Index.SampleIndexDetails getIdxDetails() {
+        return idxDetails;
+    }
+
+    protected boolean nestedLoopsApplicable(ILogicalExpression joinExpr) throws AlgebricksException {
+
+        List<LogicalVariable> usedVarList = new ArrayList<>();
+        joinExpr.getUsedVariables(usedVarList);
+        if (usedVarList.size() != 2) {
+            return false;
+        }
+
+        if (joinExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+
+        LogicalVariable var0 = usedVarList.get(0);
+        LogicalVariable var1 = usedVarList.get(1);
+
+        // Find which joinLeafInput these vars belong to.
+        // go thru the leaf inputs and see where these variables came from
+        ILogicalOperator joinLeafInput0 = joinEnum.findLeafInput(Collections.singletonList(var0));
+        if (joinLeafInput0 == null) {
+            return false; // this should not happen unless an assignment is between two joins.
+        }
+
+        ILogicalOperator joinLeafInput1 = joinEnum.findLeafInput(Collections.singletonList(var1));
+        if (joinLeafInput1 == null) {
+            return false;
+        }
+
+        // We need to find out which one of these is the inner joinLeafInput. So for that get the joinLeafInput of this join node.
+        ILogicalOperator innerLeafInput = joinEnum.joinLeafInputsHashMap.get(this.correspondingEmptyTupleSourceOp);
+
+        // This must equal one of the two joinLeafInputsHashMap found above. check for sanity!!
+        if (innerLeafInput != joinLeafInput1 && innerLeafInput != joinLeafInput0) {
+            return false; // This should not happen. So debug to find out why this happened.
+        }
+
+        if (innerLeafInput == joinLeafInput0) {
+            joinEnum.localJoinOp.getInputs().get(0).setValue(joinLeafInput1);
+        } else {
+            joinEnum.localJoinOp.getInputs().get(0).setValue(joinLeafInput0);
+        }
+
+        joinEnum.localJoinOp.getInputs().get(1).setValue(innerLeafInput);
+
+        // We will always use the first join Op to provide the joinOp input for invoking rewritePre
+        AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinEnum.localJoinOp;
+        joinOp.getCondition().setValue(joinExpr);
+
+        // Now call the rewritePre code
+        IntroduceJoinAccessMethodRule tmp = new IntroduceJoinAccessMethodRule();
+        boolean temp = tmp.checkApplicable(new MutableObject<>(joinEnum.localJoinOp), joinEnum.optCtx);
+
+        return temp;
+    }
+
+    /** one is a subset of two */
+    private boolean subset(int one, int two) {
+        return (one & two) == one;
+    }
+
+    protected void findApplicableJoinConditions() {
+        List<JoinCondition> joinConditions = joinEnum.getJoinConditions();
+
+        int i = 0;
+        for (JoinCondition jc : joinConditions) {
+            if (subset(jc.datasetBits, this.datasetBits)) {
+                this.applicableJoinConditions.add(i);
+            }
+            i++;
+        }
+    }
+
+    protected List<Integer> getNewJoinConditionsOnly() {
+        List<Integer> newJoinConditions = new ArrayList<>();
+        JoinNode leftJn = this.leftJn;
+        JoinNode rightJn = this.rightJn;
+        // find the new table being added. This assume only zig zag trees for now.
+        int newTableBits = 0;
+        if (leftJn.jnArrayIndex <= joinEnum.numberOfTerms) {
+            newTableBits = leftJn.datasetBits;
+        } else if (rightJn.jnArrayIndex <= joinEnum.numberOfTerms) {
+            newTableBits = rightJn.datasetBits;
+        }
+
+        if (LOGGER.isTraceEnabled() && newTableBits == 0) {
+            LOGGER.trace("newTable Bits == 0");
+        }
+
+        // All the new join predicates will have these bits turned on
+        for (int idx : this.applicableJoinConditions) {
+            if ((joinEnum.joinConditions.get(idx).datasetBits & newTableBits) > 0) {
+                newJoinConditions.add(idx);
+            }
+        }
+
+        return newJoinConditions; // this can be of size 0 because this may be a cartesian join
+    }
+
+    public int addSingleDatasetPlans() {
+        List<PlanNode> allPlans = joinEnum.allPlans;
+        ICost opCost, totalCost;
+
+        opCost = joinEnum.getCostMethodsHandle().costFullScan(this);
+        totalCost = opCost;
+        if (this.cheapestPlanIndex == PlanNode.NO_PLAN || opCost.costLT(this.cheapestPlanCost)) {
+            // for now just add one plan
+            PlanNode pn = new PlanNode(allPlans.size(), joinEnum);
+            pn.jn = this;
+            pn.datasetName = this.datasetNames.get(0);
+            pn.correspondingEmptyTupleSourceOp = this.correspondingEmptyTupleSourceOp;
+            pn.jnIndexes[0] = this.jnArrayIndex;
+            pn.jnIndexes[1] = JoinNode.NO_JN;
+            pn.planIndexes[0] = PlanNode.NO_PLAN; // There ane no plans below this plan.
+            pn.planIndexes[1] = PlanNode.NO_PLAN; // There ane no plans below this plan.
+            pn.opCost = opCost;
+            pn.scanOp = PlanNode.ScanMethod.TABLE_SCAN;
+            pn.totalCost = totalCost;
+
+            allPlans.add(pn);
+            this.planIndexesArray.add(allPlans.size() - 1);
+            this.cheapestPlanCost = totalCost;
+            this.cheapestPlanIndex = allPlans.size() - 1;
+            return this.cheapestPlanIndex;
+        }
+        return PlanNode.NO_PLAN;
+    }
+
+    protected void buildIndexPlan() {
+        List<PlanNode> allPlans = joinEnum.allPlans;
+        ICost opCost, totalCost;
+
+        opCost = joinEnum.getCostMethodsHandle().costIndexScan(this);
+        totalCost = opCost;
+        if (this.cheapestPlanIndex == PlanNode.NO_PLAN || opCost.costLT(this.cheapestPlanCost)) {
+            // for now just add one plan
+            PlanNode pn = new PlanNode(allPlans.size(), joinEnum);
+            pn.jn = this;
+            pn.datasetName = this.datasetNames.get(0);
+            pn.correspondingEmptyTupleSourceOp = this.correspondingEmptyTupleSourceOp;
+            pn.jnIndexes[0] = this.jnArrayIndex;
+            pn.jnIndexes[1] = JoinNode.NO_JN;
+            pn.planIndexes[0] = PlanNode.NO_PLAN; // There ane no plans below this plan.
+            pn.planIndexes[1] = PlanNode.NO_PLAN; // There ane no plans below this plan.
+            pn.opCost = opCost;
+            pn.scanOp = PlanNode.ScanMethod.INDEX_SCAN;
+            pn.totalCost = totalCost;
+
+            allPlans.add(pn);
+            this.planIndexesArray.add(allPlans.size() - 1);
+            this.cheapestPlanCost = totalCost;
+            this.cheapestPlanIndex = allPlans.size() - 1;
+        }
+    }
+
+    protected void costAndChooseIndexPlans(ILogicalOperator leafInput,
+            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) throws AlgebricksException {
+        // Skip indexes with selectivity greater than 0.1, add the SKIP_SECONDARY_INDEX annotation to its expression.
+        double sel;
+        Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
+        while (amIt.hasNext()) {
+            Map.Entry<IAccessMethod, AccessMethodAnalysisContext> amEntry = amIt.next();
+            AccessMethodAnalysisContext analysisCtx = amEntry.getValue();
+            Iterator<Map.Entry<Index, List<Pair<Integer, Integer>>>> indexIt =
+                    analysisCtx.getIteratorForIndexExprsAndVars();
+            List<IOptimizableFuncExpr> exprs = analysisCtx.getMatchedFuncExprs();
+            int exprIndex = 0;
+            while (indexIt.hasNext()) {
+                Map.Entry<Index, List<Pair<Integer, Integer>>> indexEntry = indexIt.next();
+                Index chosenIndex = indexEntry.getKey();
+                IOptimizableFuncExpr expr = exprs.get(exprIndex++);
+                AbstractFunctionCallExpression afce = expr.getFuncExpr();
+                PredicateCardinalityAnnotation selectivityAnnotation =
+                        afce.getAnnotation(PredicateCardinalityAnnotation.class);
+                if (selectivityAnnotation != null) {
+                    sel = selectivityAnnotation.getSelectivity();
+                    if (sel >= joinEnum.stats.SELECTIVITY_FOR_SECONDARY_INDEX_SELECTION) {
+                        afce.putAnnotation(SkipSecondaryIndexSearchExpressionAnnotation
+                                .newInstance(Collections.singleton(chosenIndex.getIndexName())));
+                    } else {
+                        buildIndexPlan();
+                    }
+                }
+            }
+        }
+    }
+
+    public void addIndexAccessPlans(ILogicalOperator leafInput) throws AlgebricksException {
+        IntroduceSelectAccessMethodRule tmp = new IntroduceSelectAccessMethodRule();
+        List<Pair<IAccessMethod, Index>> chosenIndexes = new ArrayList<>();
+        Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = new TreeMap<>();
+        boolean index_access_possible =
+                tmp.checkApplicable(new MutableObject<>(leafInput), joinEnum.optCtx, chosenIndexes, analyzedAMs);
+        this.chosenIndexes = chosenIndexes;
+        this.analyzedAMs = analyzedAMs;
+        if (index_access_possible) {
+            costAndChooseIndexPlans(leafInput, analyzedAMs);
+        }
+    }
+
+    protected int buildHashJoinPlan(JoinNode leftJn, JoinNode rightJn, ILogicalExpression hashJoinExpr) {
+        List<PlanNode> allPlans = joinEnum.allPlans;
+        PlanNode pn;
+        ICost hjCost, childCosts, totalCost;
+
+        this.leftJn = leftJn;
+        this.rightJn = rightJn;
+        int leftPlan = leftJn.cheapestPlanIndex;
+        int rightPlan = rightJn.cheapestPlanIndex;
+
+        if (hashJoinExpr == null || hashJoinExpr == ConstantExpression.TRUE) {
+            return PlanNode.NO_PLAN;
+        }
+
+        if (joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_LEFTDEEP)
+                && !leftJn.IsBaseLevelJoinNode()) {
+            return PlanNode.NO_PLAN;
+        }
+
+        if (joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_RIGHTDEEP)
+                && !rightJn.IsBaseLevelJoinNode()) {
+            return PlanNode.NO_PLAN;
+        }
+
+        if (rightJn.cardinality * rightJn.size <= leftJn.cardinality * leftJn.size || joinEnum.forceJoinOrderMode
+                || !joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_ZIGZAG)) {
+            // We want to build with the smaller side.
+            hjCost = joinEnum.getCostMethodsHandle().costHashJoin(this);
+            childCosts = allPlans.get(leftPlan).totalCost.costAdd(allPlans.get(rightPlan).totalCost);
+            totalCost = hjCost.costAdd(childCosts);
+            if (this.cheapestPlanIndex == PlanNode.NO_PLAN || totalCost.costLT(this.cheapestPlanCost)) {
+                pn = new PlanNode(allPlans.size(), joinEnum);
+                pn.jn = this;
+                pn.jnIndexes[0] = leftJn.jnArrayIndex;
+                pn.jnIndexes[1] = rightJn.jnArrayIndex;
+                pn.planIndexes[0] = leftPlan;
+                pn.planIndexes[1] = rightPlan;
+                pn.joinOp = PlanNode.JoinMethod.HYBRID_HASH_JOIN; // need to check that all the conditions have equality predicates ONLY.
+                pn.side = HashJoinExpressionAnnotation.BuildSide.RIGHT;
+                pn.joinExpr = hashJoinExpr;
+                pn.opCost = hjCost;
+                pn.totalCost = totalCost;
+                pn.leftExchangeCost = joinEnum.getCostMethodsHandle().computeHJProbeExchangeCost(this);
+                pn.rightExchangeCost = joinEnum.getCostMethodsHandle().computeHJBuildExchangeCost(this);
+
+                allPlans.add(pn);
+                this.planIndexesArray.add(allPlans.size() - 1);
+                this.cheapestPlanCost = totalCost;
+                this.cheapestPlanIndex = allPlans.size() - 1;
+                return this.cheapestPlanIndex;
+            }
+        }
+
+        return PlanNode.NO_PLAN;
+    }
+
+    protected int buildBroadcastHashJoinPlan(JoinNode leftJn, JoinNode rightJn, ILogicalExpression hashJoinExpr) {
+        List<PlanNode> allPlans = joinEnum.allPlans;
+        PlanNode pn;
+        ICost bcastHjCost, childCosts, totalCost;
+
+        this.leftJn = leftJn;
+        this.rightJn = rightJn;
+        int leftPlan = leftJn.cheapestPlanIndex;
+        int rightPlan = rightJn.cheapestPlanIndex;
+
+        if (hashJoinExpr == null || hashJoinExpr == ConstantExpression.TRUE) {
+            return PlanNode.NO_PLAN;
+        }
+
+        if (joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_LEFTDEEP)
+                && !leftJn.IsBaseLevelJoinNode()) {
+            return PlanNode.NO_PLAN;
+        }
+
+        if (joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_RIGHTDEEP)
+                && !rightJn.IsBaseLevelJoinNode()) {
+            return PlanNode.NO_PLAN;
+        }
+
+        if (rightJn.cardinality * rightJn.size <= leftJn.cardinality * leftJn.size || joinEnum.forceJoinOrderMode
+                || !joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_ZIGZAG)) {
+            // We want to broadcast and build with the smaller side.
+            bcastHjCost = joinEnum.getCostMethodsHandle().costBroadcastHashJoin(this);
+            childCosts = allPlans.get(leftPlan).totalCost.costAdd(allPlans.get(rightPlan).totalCost);
+            totalCost = bcastHjCost.costAdd(childCosts);
+            if (this.cheapestPlanIndex == PlanNode.NO_PLAN || totalCost.costLT(this.cheapestPlanCost)) {
+                pn = new PlanNode(allPlans.size(), joinEnum);
+                pn.jn = this;
+                pn.jnIndexes[0] = leftJn.jnArrayIndex;
+                pn.jnIndexes[1] = rightJn.jnArrayIndex;
+                pn.planIndexes[0] = leftPlan;
+                pn.planIndexes[1] = rightPlan;
+                pn.joinOp = PlanNode.JoinMethod.BROADCAST_HASH_JOIN; // need to check that all the conditions have equality predicates ONLY.
+                pn.side = HashJoinExpressionAnnotation.BuildSide.RIGHT;
+                pn.joinExpr = hashJoinExpr;
+                pn.opCost = bcastHjCost;
+                pn.totalCost = totalCost;
+                pn.leftExchangeCost = joinEnum.getCostHandle().zeroCost();
+                pn.rightExchangeCost = joinEnum.getCostMethodsHandle().computeBHJBuildExchangeCost(this);
+
+                allPlans.add(pn);
+                this.planIndexesArray.add(allPlans.size() - 1);
+                this.cheapestPlanCost = totalCost;
+                this.cheapestPlanIndex = allPlans.size() - 1;
+                return this.cheapestPlanIndex;
+            }
+        }
+
+        return PlanNode.NO_PLAN;
+    }
+
+    protected int buildNLJoinPlan(JoinNode leftJn, JoinNode rightJn, ILogicalExpression nestedLoopJoinExpr)
+            throws AlgebricksException {
+        // Build a nested loops plan, first check if it is possible
+        // left right order must be preserved and right side should be a single data set
+        List<PlanNode> allPlans = joinEnum.allPlans;
+        int numberOfTerms = joinEnum.numberOfTerms;
+        PlanNode pn;
+        ICost nljCost, childCosts, totalCost;
+
+        this.leftJn = leftJn;
+        this.rightJn = rightJn;
+        int leftPlan = leftJn.cheapestPlanIndex;
+        int rightPlan = rightJn.cheapestPlanIndex;
+        if (rightJn.jnArrayIndex > numberOfTerms) {
+            // right side consists of more than one table
+            return PlanNode.NO_PLAN; // nested loop plan not possible.
+        }
+
+        if (nestedLoopJoinExpr == null || !rightJn.nestedLoopsApplicable(nestedLoopJoinExpr)) {
+            return PlanNode.NO_PLAN;
+        }
+
+        nljCost = joinEnum.getCostMethodsHandle().costIndexNLJoin(this);
+        childCosts = allPlans.get(leftPlan).totalCost;
+        totalCost = nljCost.costAdd(childCosts);
+        if (this.cheapestPlanIndex == PlanNode.NO_PLAN || totalCost.costLT(this.cheapestPlanCost)) {
+            pn = new PlanNode(allPlans.size(), joinEnum);
+            pn.jn = this;
+            pn.jnIndexes[0] = leftJn.jnArrayIndex;
+            pn.jnIndexes[1] = rightJn.jnArrayIndex;
+            pn.planIndexes[0] = leftPlan;
+            pn.planIndexes[1] = rightPlan;
+            pn.joinOp = PlanNode.JoinMethod.INDEX_NESTED_LOOP_JOIN;
+            pn.joinExpr = nestedLoopJoinExpr; // save it so can be used to add the NESTED annotation in getNewTree.
+            pn.opCost = nljCost;
+            pn.totalCost = totalCost;
+            pn.leftExchangeCost = joinEnum.getCostMethodsHandle().computeNLJOuterExchangeCost(this);
+            pn.rightExchangeCost = joinEnum.getCostHandle().zeroCost();
+
+            allPlans.add(pn);
+            this.planIndexesArray.add(allPlans.size() - 1);
+            this.cheapestPlanCost = totalCost;
+            this.cheapestPlanIndex = allPlans.size() - 1;
+            return allPlans.size() - 1;
+        }
+        return PlanNode.NO_PLAN;
+    }
+
+    protected int buildCPJoinPlan(JoinNode leftJn, JoinNode rightJn, ILogicalExpression hashJoinExpr,
+            ILogicalExpression nestedLoopJoinExpr) {
+        // Now build a cartesian product nested loops plan
+        List<PlanNode> allPlans = joinEnum.allPlans;
+        PlanNode pn;
+        ICost cpCost, childCosts, totalCost;
+
+        this.leftJn = leftJn;
+        this.rightJn = rightJn;
+        int leftPlan = leftJn.cheapestPlanIndex;
+        int rightPlan = rightJn.cheapestPlanIndex;
+
+        ILogicalExpression cpJoinExpr = null;
+        List<Integer> newJoinConditions = this.getNewJoinConditionsOnly();
+        if (hashJoinExpr == null && nestedLoopJoinExpr == null) {
+            cpJoinExpr = joinEnum.combineAllConditions(newJoinConditions);
+        } else if (hashJoinExpr != null && nestedLoopJoinExpr == null) {
+            cpJoinExpr = hashJoinExpr;
+        } else if (hashJoinExpr == null && nestedLoopJoinExpr != null) {
+            cpJoinExpr = nestedLoopJoinExpr;
+        } else if (Objects.equals(hashJoinExpr, nestedLoopJoinExpr) == true) {
+            cpJoinExpr = hashJoinExpr;
+        } else if (Objects.equals(hashJoinExpr, nestedLoopJoinExpr) == false) {
+            ScalarFunctionCallExpression andExpr = new ScalarFunctionCallExpression(
+                    BuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.AND));
+            andExpr.getArguments().add(new MutableObject<>(hashJoinExpr));
+            andExpr.getArguments().add(new MutableObject<>(nestedLoopJoinExpr));
+            cpJoinExpr = andExpr;
+        }
+
+        cpCost = joinEnum.getCostMethodsHandle().costCartesianProductJoin(this);
+        childCosts = allPlans.get(leftPlan).totalCost.costAdd(allPlans.get(rightPlan).totalCost);
+        totalCost = cpCost.costAdd(childCosts);
+        if (this.cheapestPlanIndex == PlanNode.NO_PLAN || totalCost.costLT(this.cheapestPlanCost)) {
+            pn = new PlanNode(allPlans.size(), joinEnum);
+            pn.jn = this;
+            pn.jnIndexes[0] = leftJn.jnArrayIndex;
+            pn.jnIndexes[1] = rightJn.jnArrayIndex;
+            pn.planIndexes[0] = leftPlan;
+            pn.planIndexes[1] = rightPlan;
+            pn.joinOp = PlanNode.JoinMethod.CARTESIAN_PRODUCT_JOIN;
+            pn.joinExpr = Objects.requireNonNullElse(cpJoinExpr, ConstantExpression.TRUE);
+            pn.opCost = cpCost;
+            pn.totalCost = totalCost;
+            pn.leftExchangeCost = joinEnum.getCostHandle().zeroCost();
+            pn.rightExchangeCost = joinEnum.getCostMethodsHandle().computeCPRightExchangeCost(this);
+
+            allPlans.add(pn);
+            this.planIndexesArray.add(allPlans.size() - 1);
+            this.cheapestPlanCost = totalCost;
+            this.cheapestPlanIndex = allPlans.size() - 1;
+            return allPlans.size() - 1;
+        }
+        return PlanNode.NO_PLAN;
+    }
+
+    protected Pair<Integer, ICost> addMultiDatasetPlans(JoinNode leftJn, JoinNode rightJn, int level)
+            throws AlgebricksException {
+        this.leftJn = leftJn;
+        this.rightJn = rightJn;
+        ICost noJoinCost = joinEnum.getCostHandle().maxCost();
+
+        if (leftJn.planIndexesArray.size() == 0 || rightJn.planIndexesArray.size() == 0) {
+            return new Pair<>(PlanNode.NO_PLAN, noJoinCost);
+        }
+
+        if (this.cardinality >= Cost.MAX_CARD) {
+            return new Pair<>(PlanNode.NO_PLAN, noJoinCost); // no card hint available, so do not add this plan
+        }
+
+        List<Integer> newJoinConditions = this.getNewJoinConditionsOnly(); // these will be a subset of applicable join conditions.
+        ILogicalExpression hashJoinExpr = joinEnum.getHashJoinExpr(newJoinConditions);
+        ILogicalExpression nestedLoopJoinExpr = joinEnum.getNestedLoopJoinExpr(newJoinConditions);
+
+        if ((newJoinConditions.size() == 0) && joinEnum.connectedJoinGraph) {
+            // at least one plan must be there at each level as the graph is fully connected.
+            if (leftJn.cardinality * rightJn.cardinality > 10000.0) {
+                return new Pair<>(PlanNode.NO_PLAN, noJoinCost);
+            }
+        }
+
+        double current_card = this.cardinality;
+        if (current_card >= Cost.MAX_CARD) {
+            return new Pair<>(PlanNode.NO_PLAN, noJoinCost); // no card hint available, so do not add this plan
+        }
+
+        int hjPlan, commutativeHjPlan, bcastHjPlan, commutativeBcastHjPlan, nljPlan, commutativeNljPlan, cpPlan,
+                commutativeCpPlan;
+        hjPlan = commutativeHjPlan = bcastHjPlan =
+                commutativeBcastHjPlan = nljPlan = commutativeNljPlan = cpPlan = commutativeCpPlan = PlanNode.NO_PLAN;
+
+        HashJoinExpressionAnnotation.BuildSide hintHashJoin = joinEnum.findHashJoinHint(newJoinConditions);
+        BroadcastExpressionAnnotation.BroadcastSide hintBroadcastHashJoin = null;
+        boolean hintNLJoin = false;
+        if (hintHashJoin == null) {
+            hintBroadcastHashJoin = joinEnum.findBroadcastHashJoinHint(newJoinConditions);
+            if (hintBroadcastHashJoin == null) {
+                hintNLJoin = joinEnum.findNLJoinHint(newJoinConditions);
+            }
+        }
+
+        if (leftJn.cheapestPlanIndex == PlanNode.NO_PLAN || rightJn.cheapestPlanIndex == PlanNode.NO_PLAN) {
+            return new Pair<>(PlanNode.NO_PLAN, noJoinCost);
+        }
+
+        if (hintHashJoin != null) {
+            hjPlan = buildHashJoinPlan(leftJn, rightJn, hashJoinExpr);
+            if (!joinEnum.forceJoinOrderMode && hintHashJoin != HashJoinExpressionAnnotation.BuildSide.RIGHT) {
+                commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, hashJoinExpr);
+            }
+            if (hjPlan == PlanNode.NO_PLAN && commutativeHjPlan == PlanNode.NO_PLAN) {
+                // Hints are attached to predicates, so newJoinConditions should not be empty, but adding the check to be safe.
+                if (!joinEnum.getJoinConditions().isEmpty() && !newJoinConditions.isEmpty()) {
+                    IWarningCollector warningCollector = joinEnum.optCtx.getWarningCollector();
+                    if (warningCollector.shouldWarn()) {
+                        warningCollector.warn(Warning.of(
+                                joinEnum.getJoinConditions().get(newJoinConditions.get(0)).joinCondition
+                                        .getSourceLocation(),
+                                ErrorCode.INAPPLICABLE_HINT, "Hash join hint not applicable and was ignored"));
+                    }
+                }
+                bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, hashJoinExpr);
+                if (!joinEnum.forceJoinOrderMode) {
+                    commutativeBcastHjPlan = buildBroadcastHashJoinPlan(rightJn, leftJn, hashJoinExpr);
+                }
+                nljPlan = buildNLJoinPlan(leftJn, rightJn, nestedLoopJoinExpr);
+                if (!joinEnum.forceJoinOrderMode) {
+                    commutativeNljPlan = buildNLJoinPlan(rightJn, leftJn, nestedLoopJoinExpr);
+                }
+                cpPlan = buildCPJoinPlan(leftJn, rightJn, hashJoinExpr, nestedLoopJoinExpr);
+                if (!joinEnum.forceJoinOrderMode) {
+                    commutativeCpPlan = buildCPJoinPlan(rightJn, leftJn, hashJoinExpr, nestedLoopJoinExpr);
+                }
+            }
+        } else if (hintBroadcastHashJoin != null) {
+            bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, hashJoinExpr);
+            if (!joinEnum.forceJoinOrderMode
+                    && hintBroadcastHashJoin != BroadcastExpressionAnnotation.BroadcastSide.RIGHT) {
+                commutativeBcastHjPlan = buildBroadcastHashJoinPlan(rightJn, leftJn, hashJoinExpr);
+            }
+            if (bcastHjPlan == PlanNode.NO_PLAN && commutativeBcastHjPlan == PlanNode.NO_PLAN) {
+                // Hints are attached to predicates, so newJoinConditions should not be empty, but adding the check to be safe.
+                if (!joinEnum.getJoinConditions().isEmpty() && !newJoinConditions.isEmpty()) {
+                    IWarningCollector warningCollector = joinEnum.optCtx.getWarningCollector();
+                    if (warningCollector.shouldWarn()) {
+                        warningCollector.warn(Warning.of(
+                                joinEnum.getJoinConditions().get(newJoinConditions.get(0)).joinCondition
+                                        .getSourceLocation(),
+                                ErrorCode.INAPPLICABLE_HINT,
+                                "Broadcast hash join hint not applicable and was ignored"));
+                    }
+                }
+
+                hjPlan = buildHashJoinPlan(leftJn, rightJn, hashJoinExpr);
+                if (!joinEnum.forceJoinOrderMode) {
+                    commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, hashJoinExpr);
+                }
+                nljPlan = buildNLJoinPlan(leftJn, rightJn, nestedLoopJoinExpr);
+                if (!joinEnum.forceJoinOrderMode) {
+                    commutativeNljPlan = buildNLJoinPlan(rightJn, leftJn, nestedLoopJoinExpr);
+                }
+                cpPlan = buildCPJoinPlan(leftJn, rightJn, hashJoinExpr, nestedLoopJoinExpr);
+                if (!joinEnum.forceJoinOrderMode) {
+                    commutativeCpPlan = buildCPJoinPlan(rightJn, leftJn, hashJoinExpr, nestedLoopJoinExpr);
+                }
+            }
+        } else if (hintNLJoin) {
+            nljPlan = buildNLJoinPlan(leftJn, rightJn, nestedLoopJoinExpr);
+            if (!joinEnum.forceJoinOrderMode) {
+                commutativeNljPlan = buildNLJoinPlan(rightJn, leftJn, nestedLoopJoinExpr);
+            }
+            if (nljPlan == PlanNode.NO_PLAN && commutativeNljPlan == PlanNode.NO_PLAN) {
+                // Hints are attached to predicates, so newJoinConditions should not be empty, but adding the check to be safe.
+                if (!joinEnum.getJoinConditions().isEmpty() && !newJoinConditions.isEmpty()) {
+                    IWarningCollector warningCollector = joinEnum.optCtx.getWarningCollector();
+                    if (warningCollector.shouldWarn()) {
+                        warningCollector.warn(Warning.of(
+                                joinEnum.getJoinConditions().get(newJoinConditions.get(0)).joinCondition
+                                        .getSourceLocation(),
+                                ErrorCode.INAPPLICABLE_HINT, "Index nested join hint not applicable and was ignored"));
+                    }
+                }
+                hjPlan = buildHashJoinPlan(leftJn, rightJn, hashJoinExpr);
+                if (!joinEnum.forceJoinOrderMode) {
+                    commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, hashJoinExpr);
+                }
+                bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, hashJoinExpr);
+                if (!joinEnum.forceJoinOrderMode) {
+                    commutativeBcastHjPlan = buildBroadcastHashJoinPlan(rightJn, leftJn, hashJoinExpr);
+                }
+                cpPlan = buildCPJoinPlan(leftJn, rightJn, hashJoinExpr, nestedLoopJoinExpr);
+                if (!joinEnum.forceJoinOrderMode) {
+                    commutativeCpPlan = buildCPJoinPlan(rightJn, leftJn, hashJoinExpr, nestedLoopJoinExpr);
+                }
+            }
+        } else {
+            hjPlan = buildHashJoinPlan(leftJn, rightJn, hashJoinExpr);
+            if (!joinEnum.forceJoinOrderMode) {
+                commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, hashJoinExpr);
+            }
+            bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, hashJoinExpr);
+            if (!joinEnum.forceJoinOrderMode) {
+                commutativeBcastHjPlan = buildBroadcastHashJoinPlan(rightJn, leftJn, hashJoinExpr);
+            }
+            nljPlan = buildNLJoinPlan(leftJn, rightJn, nestedLoopJoinExpr);
+            if (!joinEnum.forceJoinOrderMode) {
+                commutativeNljPlan = buildNLJoinPlan(rightJn, leftJn, nestedLoopJoinExpr);
+            }
+            cpPlan = buildCPJoinPlan(leftJn, rightJn, hashJoinExpr, nestedLoopJoinExpr);
+            if (!joinEnum.forceJoinOrderMode) {
+                commutativeCpPlan = buildCPJoinPlan(rightJn, leftJn, hashJoinExpr, nestedLoopJoinExpr);
+            }
+        }
+
+        if (hjPlan == PlanNode.NO_PLAN && commutativeHjPlan == PlanNode.NO_PLAN && bcastHjPlan == PlanNode.NO_PLAN
+                && commutativeBcastHjPlan == PlanNode.NO_PLAN && nljPlan == PlanNode.NO_PLAN
+                && commutativeNljPlan == PlanNode.NO_PLAN && cpPlan == PlanNode.NO_PLAN
+                && commutativeCpPlan == PlanNode.NO_PLAN) {
+            return new Pair<>(PlanNode.NO_PLAN, noJoinCost);
+        }
+
+        //Reset as these might have changed when we tried the commutative joins.
+        this.leftJn = leftJn;
+        this.rightJn = rightJn;
+
+        return new Pair<>(this.cheapestPlanIndex, this.cheapestPlanCost);
+    }
+
+    @Override
+    public String toString() {
+        if (planIndexesArray.isEmpty()) {
+            return "";
+        }
+        List<PlanNode> allPlans = joinEnum.allPlans;
+        StringBuilder sb = new StringBuilder(128);
+        // This will avoid printing JoinNodes that have no plans
+        sb.append("Printing Join Node ").append(jnArrayIndex).append('\n');
+        sb.append("datasetNames ").append('\n');
+        for (int j = 0; j < datasetNames.size(); j++) {
+            // Need to not print newline
+            sb.append(datasetNames.get(j)).append(' ');
+        }
+        sb.append("datasetIndex ").append('\n');
+        for (int j = 0; j < datasetIndexes.size(); j++) {
+            sb.append(j).append(datasetIndexes.get(j)).append('\n');
+        }
+        sb.append("datasetBits is ").append(datasetBits).append('\n');
+        if (IsBaseLevelJoinNode()) {
+            sb.append("orig cardinality is ").append((double) Math.round(origCardinality * 100) / 100).append('\n');
+        }
+        sb.append("cardinality is ").append((double) Math.round(cardinality * 100) / 100).append('\n');
+        if (planIndexesArray.size() == 0) {
+            sb.append("No plans considered for this join node").append('\n');
+        }
+        for (int j = 0; j < planIndexesArray.size(); j++) {
+            int k = planIndexesArray.get(j);
+            PlanNode pn = allPlans.get(k);
+            sb.append("planIndexesArray  [").append(j).append("] is ").append(k).append('\n');
+            sb.append("Printing PlanNode ").append(k);
+            if (IsBaseLevelJoinNode()) {
+                sb.append("DATA_SOURCE_SCAN").append('\n');
+            } else {
+                sb.append("\n");
+                sb.append(pn.joinMethod().getFirst()).append('\n');
+                sb.append("Printing Join expr ").append('\n');
+                if (pn.joinExpr != null) {
+                    sb.append(pn.joinExpr).append('\n');
+                } else {
+                    sb.append("null").append('\n');
+                }
+            }
+            sb.append("card ").append((double) Math.round(cardinality * 100) / 100).append('\n');
+            sb.append("operator cost ").append(pn.opCost.computeTotalCost()).append('\n');
+            sb.append("total cost ").append(pn.totalCost.computeTotalCost()).append('\n');
+            sb.append("jnIndexes ").append(pn.jnIndexes[0]).append(" ").append(pn.jnIndexes[1]).append('\n');
+            if (IsHigherLevelJoinNode()) {
+                PlanNode leftPlan = pn.getLeftPlanNode();
+                PlanNode rightPlan = pn.getRightPlanNode();
+                int l = leftPlan.allPlansIndex;
+                int r = rightPlan.allPlansIndex;
+                sb.append("planIndexes ").append(l).append(" ").append(r).append('\n');
+                sb.append("(lcost = ").append(leftPlan.totalCost.computeTotalCost()).append(") (rcost = ")
+                        .append(rightPlan.totalCost.computeTotalCost()).append(")").append('\n');
+            }
+            sb.append("\n");
+        }
+        sb.append("jnIndex ").append(jnIndex).append('\n');
+        sb.append("datasetBits ").append(datasetBits).append('\n');
+        sb.append("cardinality ").append((double) Math.round(cardinality * 100) / 100).append('\n');
+        sb.append("size ").append((double) Math.round(size * 100) / 100).append('\n');
+        sb.append("level ").append(level).append('\n');
+        sb.append("highestDatasetId ").append(highestDatasetId).append('\n');
+        sb.append("--------------------------------------").append('\n');
+        return sb.toString();
+    }
+
+    public void printCostOfAllPlans(StringBuilder sb) {
+        List<PlanNode> allPlans = joinEnum.allPlans;
+        for (int planIndex : planIndexesArray) {
+            sb.append("plan ").append(planIndex).append(" cost is ")
+                    .append(allPlans.get(planIndex).totalCost.computeTotalCost()).append('\n');
+        }
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/PlanNode.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/PlanNode.java
new file mode 100644
index 0000000..1dca15f
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/PlanNode.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.cbo;
+
+import org.apache.asterix.optimizer.cost.ICost;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.HashJoinExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+
+public class PlanNode {
+
+    public static int NO_PLAN = -1;
+
+    private final JoinEnum joinEnum;
+    int allPlansIndex;
+    int[] planIndexes;
+    int[] jnIndexes;
+    JoinNode jn;
+    String datasetName;
+    ICost opCost;
+    ICost totalCost;
+    ICost leftExchangeCost;
+    ICost rightExchangeCost;
+    JoinMethod joinOp;
+    // Used to indicate which side to build for HJ and which side to broadcast for BHJ.
+    HashJoinExpressionAnnotation.BuildSide side;
+    ScanMethod scanOp;
+    ILogicalExpression joinExpr;
+    DataSourceScanOperator correspondingDataSourceScanOp;
+    EmptyTupleSourceOperator correspondingEmptyTupleSourceOp;
+
+    public enum ScanMethod {
+        INDEX_SCAN,
+        TABLE_SCAN
+    }
+
+    public enum JoinMethod {
+        HYBRID_HASH_JOIN,
+        BROADCAST_HASH_JOIN,
+        INDEX_NESTED_LOOP_JOIN,
+        CARTESIAN_PRODUCT_JOIN
+    }
+
+    public PlanNode(int planIndex, JoinEnum joinE) {
+        this.allPlansIndex = planIndex;
+        joinEnum = joinE;
+        planIndexes = new int[2]; // 0 is for left, 1 is for right
+        jnIndexes = new int[2]; // join node index(es)
+    }
+
+    public int[] getPlanIndexes() {
+        return planIndexes;
+    }
+
+    public int getLeftPlanIndex() {
+        return planIndexes[0];
+    }
+
+    public PlanNode getLeftPlanNode() {
+        if (planIndexes[0] == NO_PLAN) {
+            return null;
+        }
+        return joinEnum.allPlans.get(planIndexes[0]);
+    }
+
+    public PlanNode getRightPlanNode() {
+        if (planIndexes[1] == NO_PLAN) {
+            return null;
+        }
+        return joinEnum.allPlans.get(planIndexes[1]);
+    }
+
+    public JoinNode getJoinNode() {
+        return jn;
+    }
+
+    public int getRightPlanIndex() {
+        return planIndexes[1];
+    }
+
+    public int getLeftJoinIndex() {
+        return jnIndexes[0];
+    }
+
+    public int getRightJoinIndex() {
+        return jnIndexes[1];
+    }
+
+    public boolean IsScanNode() {
+        return getLeftPlanIndex() == NO_PLAN && getRightPlanIndex() == NO_PLAN;
+    }
+
+    public boolean IsJoinNode() {
+        return getLeftPlanIndex() != NO_PLAN && getRightPlanIndex() != NO_PLAN;
+    }
+
+    Pair<String, String> joinMethod() {
+        if (this.joinOp == PlanNode.JoinMethod.HYBRID_HASH_JOIN) {
+            return new Pair<>("HASH JOIN", "HJ");
+        } else if (this.joinOp == PlanNode.JoinMethod.BROADCAST_HASH_JOIN) {
+            return new Pair<>("BROADCAST HASH JOIN", "BHJ");
+        } else if (this.joinOp == PlanNode.JoinMethod.INDEX_NESTED_LOOP_JOIN) {
+            return new Pair<>("INDEX NESTED LOOPS JOIN", "INLJ");
+        } else if (this.joinOp == PlanNode.JoinMethod.CARTESIAN_PRODUCT_JOIN) {
+            return new Pair<>("CARTESIAN PRODUCT JOIN", "CPJ");
+        }
+        return new Pair<>("", "");
+    }
+
+    public String getDatasetName() {
+        return datasetName;
+    }
+
+    public DataSourceScanOperator getDataSourceScanOp() {
+        return correspondingDataSourceScanOp; // This applies only to singleDataSetPlans
+    }
+
+    public EmptyTupleSourceOperator getEmptyTupleSourceOp() {
+        return correspondingEmptyTupleSourceOp; // This applies only to singleDataSetPlans
+    }
+
+    public ICost getOpCost() {
+        return opCost;
+    }
+
+    public double computeOpCost() {
+        return opCost.computeTotalCost();
+    }
+
+    public ICost getTotalCost() {
+        return totalCost;
+    }
+
+    public double getLeftExchangeCost() {
+        return leftExchangeCost.computeTotalCost();
+    }
+
+    public double getRightExchangeCost() {
+        return rightExchangeCost.computeTotalCost();
+    }
+
+    public double computeTotalCost() {
+        return totalCost.computeTotalCost();
+    }
+
+    public ScanMethod getScanOp() {
+        return scanOp;
+    }
+
+    public JoinMethod getJoinOp() {
+        return joinOp;
+    }
+
+    public ILogicalExpression getJoinExpr() {
+        return joinExpr;
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/Stats.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/Stats.java
new file mode 100644
index 0000000..96bc412
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/cbo/Stats.java
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.cbo;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.declared.DataSource;
+import org.apache.asterix.metadata.declared.DataSourceId;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Index;
+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.expressions.JoinProductivityAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.PredicateCardinalityAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+
+public class Stats {
+
+    public double SELECTIVITY_FOR_SECONDARY_INDEX_SELECTION = 0.1;
+
+    protected IOptimizationContext optCtx;
+    protected JoinEnum joinEnum;
+
+    public Stats(IOptimizationContext context, JoinEnum joinE) {
+        optCtx = context;
+        joinEnum = joinE;
+    }
+
+    public DataverseName findDataverseName(DataSourceScanOperator scanOp) {
+        if (scanOp == null) {
+            // this should rarely happen (IN lists may cause this)
+            return null;
+        }
+        DataSourceId dsid = (DataSourceId) scanOp.getDataSource().getId();
+        return dsid.getDataverseName();
+    }
+
+    public Index findSampleIndex(DataSourceScanOperator scanOp, IOptimizationContext context)
+            throws AlgebricksException {
+        DataverseName dataverseName = findDataverseName(scanOp);
+        DataSource ds = (DataSource) scanOp.getDataSource();
+        DataSourceId dsid = ds.getId();
+        MetadataProvider mdp = (MetadataProvider) context.getMetadataProvider();
+        return mdp.findSampleIndex(dataverseName, dsid.getDatasourceName());
+    }
+
+    private double findJoinSelectivity(JoinProductivityAnnotation anno, AbstractFunctionCallExpression joinExpr)
+            throws AlgebricksException {
+        List<LogicalVariable> exprUsedVars = new ArrayList<>();
+        joinExpr.getUsedVariables(exprUsedVars);
+        if (exprUsedVars.size() != 2) {
+            // Since there is a left and right dataset here, expecting only two variables.
+            return 1.0;
+        }
+        int idx1 = joinEnum.findJoinNodeIndex(exprUsedVars.get(0)) + 1;
+        int idx2 = joinEnum.findJoinNodeIndex(exprUsedVars.get(1)) + 1;
+        double card1 = joinEnum.getJnArray()[idx1].origCardinality;
+        double card2 = joinEnum.getJnArray()[idx2].origCardinality;
+        if (card1 == 0.0 || card2 == 0.0) // should not happen
+        {
+            return 1.0;
+        }
+
+        // join sel  = leftside * productivity/(card1 * card2);
+        if (anno != null) {
+            int leftIndex = joinEnum.findJoinNodeIndexByName(anno.getLeftSideDataSet());
+            if (leftIndex != idx1 && leftIndex != idx2) {
+                // should not happen
+                return 1.0;
+            }
+            if (leftIndex == idx1) {
+                return anno.getJoinProductivity() / card2;
+            } else {
+                return anno.getJoinProductivity() / card1;
+            }
+        } else {
+            if (card1 < card2) {
+                // we are assuming that the smaller side is the primary side and that the join is Pk-Fk join.
+                return 1.0 / card1;
+            }
+            return 1.0 / card2;
+        }
+    }
+
+    // The expression we get may not be a base condition. It could be comprised of ors and ands and nots. So have to
+    //recursively find the overall selectivity.
+    protected double getSelectivityFromAnnotation(AbstractFunctionCallExpression afcExpr, boolean join)
+            throws AlgebricksException {
+        double sel = 1.0;
+
+        if (afcExpr.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.OR)) {
+            double orSel = getSelectivityFromAnnotation(
+                    (AbstractFunctionCallExpression) afcExpr.getArguments().get(0).getValue(), join);
+            for (int i = 1; i < afcExpr.getArguments().size(); i++) {
+                ILogicalExpression lexpr = afcExpr.getArguments().get(i).getValue();
+                if (lexpr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+                    sel = getSelectivityFromAnnotation(
+                            (AbstractFunctionCallExpression) afcExpr.getArguments().get(i).getValue(), join);
+                    orSel = orSel + sel - orSel * sel;
+                }
+            }
+            return orSel;
+        } else if (afcExpr.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.AND)) {
+            double andSel = 1.0;
+            for (int i = 0; i < afcExpr.getArguments().size(); i++) {
+                ILogicalExpression lexpr = afcExpr.getArguments().get(i).getValue();
+                if (lexpr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+                    sel = getSelectivityFromAnnotation(
+                            (AbstractFunctionCallExpression) afcExpr.getArguments().get(i).getValue(), join);
+                    andSel *= sel;
+                }
+            }
+            return andSel;
+        } else if (afcExpr.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.NOT)) {
+            ILogicalExpression lexpr = afcExpr.getArguments().get(0).getValue();
+            if (lexpr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+                sel = getSelectivityFromAnnotation(
+                        (AbstractFunctionCallExpression) afcExpr.getArguments().get(0).getValue(), join);
+                return 1.0 - sel;
+            }
+        }
+
+        double s = 1.0;
+        PredicateCardinalityAnnotation pca = afcExpr.getAnnotation(PredicateCardinalityAnnotation.class);
+        if (pca != null) {
+            s = pca.getSelectivity();
+            sel *= s;
+        } else {
+            JoinProductivityAnnotation jpa = afcExpr.getAnnotation(JoinProductivityAnnotation.class);
+            s = findJoinSelectivity(jpa, afcExpr);
+            sel *= s;
+        }
+        if (join && s == 1.0) {
+            // assume no selectivity was assigned
+            joinEnum.singleDatasetPreds.add(afcExpr);
+        }
+        return sel;
+    }
+
+    public double getSelectivityFromAnnotationMain(ILogicalExpression leExpr, boolean join) throws AlgebricksException {
+        double sel = 1.0;
+
+        if (leExpr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+            AbstractFunctionCallExpression afcExpr = (AbstractFunctionCallExpression) leExpr;
+            sel = getSelectivityFromAnnotation(afcExpr, join);
+        }
+
+        return sel;
+    }
+
+    // The next two routines should be combined and made more general
+    protected double getSelectivity(ILogicalOperator op, boolean join) throws AlgebricksException {
+        double sel = 1.0; // safe to return 1 if there is no annotation
+
+        if (op == null) {
+            return sel;
+        }
+
+        // find all the selectOperators here.
+        while (op.getOperatorTag() != LogicalOperatorTag.EMPTYTUPLESOURCE) {
+            if (op.getOperatorTag() == LogicalOperatorTag.SELECT) {
+                SelectOperator selOper = (SelectOperator) op;
+                sel *= getSelectivityFromAnnotationMain(selOper.getCondition().getValue(), join);
+            }
+            if (op.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+                sel *= getSelectivity((SubplanOperator) op);
+            }
+            op = op.getInputs().get(0).getValue();
+        }
+        return sel;
+    }
+
+    protected double getSelectivity(SubplanOperator subplanOp) throws AlgebricksException {
+        double sel = 1.0; // safe to return 1 if there is no annotation
+        //ILogicalOperator op = subplanOp;
+        ILogicalOperator op = subplanOp.getNestedPlans().get(0).getRoots().get(0).getValue();
+        while (true) {
+            if (op.getOperatorTag() == LogicalOperatorTag.SELECT) {
+                SelectOperator selOper = (SelectOperator) op;
+                sel *= getSelectivityFromAnnotationMain(selOper.getCondition().getValue(), false);
+            }
+            if (op.getInputs().size() > 0) {
+                op = op.getInputs().get(0).getValue();
+            } else {
+                break;
+            }
+        }
+        return sel;
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
index ac7fabe..6a24d59 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
@@ -40,6 +40,8 @@
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.statement.AnalyzeDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeStatement;
 import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
 import org.apache.asterix.lang.common.statement.CreateDataverseStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedStatement;
@@ -371,6 +373,27 @@
                             dataverseName);
                 }
                 break;
+
+            case ANALYZE:
+                AnalyzeStatement analyzeStmt = (AnalyzeStatement) stmt;
+                if (analyzeStmt.getDataverseName() != null) {
+                    dataverseName = analyzeStmt.getDataverseName();
+                }
+                invalidOperation = isMetadataDataverse(dataverseName);
+                if (invalidOperation) {
+                    message = String.format(BAD_DATAVERSE_OBJECT_DDL_MESSAGE, "analyze", dataset(), dataverseName);
+                }
+                break;
+            case ANALYZE_DROP:
+                AnalyzeDropStatement analyzeDropStmt = (AnalyzeDropStatement) stmt;
+                if (analyzeDropStmt.getDataverseName() != null) {
+                    dataverseName = analyzeDropStmt.getDataverseName();
+                }
+                invalidOperation = isMetadataDataverse(dataverseName);
+                if (invalidOperation) {
+                    message = String.format(BAD_DATAVERSE_OBJECT_DDL_MESSAGE, "analyze drop", dataset(), dataverseName);
+                }
+                break;
         }
 
         if (invalidOperation) {
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 8a91059..b47d45d 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
@@ -51,7 +51,6 @@
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.result.ResultSetId;
 
-import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 
@@ -137,6 +136,7 @@
         private Profile profile;
         private ProfileType profileType;
         private long totalWarningsCount;
+        private long compileTime;
 
         public long getCount() {
             return count;
@@ -187,10 +187,18 @@
         public void setProfileType(ProfileType profileType) {
             this.profileType = profileType;
         }
+
+        public void setCompileTime(long compileTime) {
+            this.compileTime = compileTime;
+        }
+
+        public long getCompileTime() {
+            return compileTime;
+        }
     }
 
     class Profile implements Serializable {
-        private static final long serialVersionUID = 4813321148252768375L;
+        private static final long serialVersionUID = 4813321148252768376L;
 
         private transient ObjectNode profile;
 
@@ -200,16 +208,15 @@
 
         private void writeObject(ObjectOutputStream out) throws IOException {
             ObjectMapper om = new ObjectMapper();
-            out.writeUTF(om.writeValueAsString(profile));
+            byte[] bytes = om.writeValueAsBytes(profile);
+            out.writeInt(bytes.length);
+            out.write(bytes);
         }
 
         private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
             ObjectMapper om = new ObjectMapper();
-            JsonNode inNode = om.readTree(in.readUTF());
-            if (!inNode.isObject()) {
-                throw new IOException("Deserialization error");
-            }
-            profile = (ObjectNode) inNode;
+            int length = in.readInt();
+            profile = (ObjectNode) om.readTree(in.readNBytes(length));
         }
 
         public ObjectNode getProfile() {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/ResultMetadata.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultMetadata.java
similarity index 96%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/ResultMetadata.java
rename to asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultMetadata.java
index 94360a1..78f84ff 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/ResultMetadata.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultMetadata.java
@@ -16,12 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.api.common;
+package org.apache.asterix.translator;
 
 import java.util.List;
 import java.util.Set;
 
-import org.apache.asterix.translator.SessionConfig;
 import org.apache.hyracks.api.exceptions.Warning;
 import org.apache.hyracks.api.result.IResultMetadata;
 
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 3c4537f..1cca591 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
@@ -37,6 +37,7 @@
 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.IVisitorExtension;
 import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
@@ -1029,6 +1030,13 @@
                 : super.visit(qe, tupSource);
     }
 
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visit(IVisitorExtension ve, Mutable<ILogicalOperator> arg)
+            throws CompilationException {
+        // Language extensions should create a child of this class.
+        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, "Extension dispatch not implemented!");
+    }
+
     // At this point "$x in list_expr" is a quantified expression:
     // "some $y in list_expr satisfies $x = $y"
     // Look for such quantified expression with a constant list_expr ([e1, e2, ... eN])
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index 51ede69..6f03f67 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -267,6 +267,7 @@
               <excludes combine.children="append">
                 <exclude>src/test/resources/**/results_parser_sqlpp/**</exclude>
                 <exclude>src/test/resources/**/results/**</exclude>
+                <exclude>src/test/resources/**/results_cbo/**</exclude>
                 <exclude>src/test/resources/fuzzyjoin/pub/fuzzy-join-aql*.dot</exclude>
                 <exclude>src/test/resources/fuzzyjoin/pub/fuzzy-join-aql*.json</exclude>
                 <exclude>**/data/**</exclude>
@@ -388,7 +389,7 @@
     <profile>
       <id>asterix-gerrit-asterix-app</id>
       <properties>
-        <test.excludes>**/SqlppExecutionWithCancellationTest.java,**/DmlTest.java,**/RepeatedTest.java,**/SqlppExecutionTest.java,**/AqlExecutionTest.java,**/*Compression*Test.java,**/*Ssl*Test.java</test.excludes>
+        <test.excludes>**/SqlppExecutionWithCancellationTest.java,**/DmlTest.java,**/RepeatedTest.java,**/SqlppExecutionTest.java,**/AqlExecutionTest.java,**/*Compression*Test.java,**/*Ssl*Test.java,**/Podman*.java</test.excludes>
         <itest.excludes>**/*.java</itest.excludes>
       </properties>
       <build>
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 3a9d54d..adcdcb2 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
@@ -40,6 +40,7 @@
 import org.apache.asterix.common.api.IResponsePrinter;
 import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.common.config.OptimizationConfUtil;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
@@ -54,7 +55,6 @@
 import org.apache.asterix.dataflow.data.common.MergeAggregationExpressionFactory;
 import org.apache.asterix.dataflow.data.common.MissableTypeComputer;
 import org.apache.asterix.dataflow.data.common.PartialAggregationTypeComputer;
-import org.apache.asterix.external.feed.watch.FeedActivityDetails;
 import org.apache.asterix.formats.base.IDataFormat;
 import org.apache.asterix.jobgen.QueryLogicalExpressionJobGen;
 import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
@@ -63,27 +63,18 @@
 import org.apache.asterix.lang.common.base.IRewriterFactory;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
-import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.Query;
-import org.apache.asterix.lang.common.statement.StartFeedStatement;
-import org.apache.asterix.lang.common.statement.ViewDecl;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
-import org.apache.asterix.lang.common.util.FunctionUtil;
-import org.apache.asterix.lang.sqlpp.rewrites.SqlppQueryRewriter;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.optimizer.base.AsterixOptimizationContext;
-import org.apache.asterix.optimizer.base.FuzzyUtils;
-import org.apache.asterix.optimizer.rules.DisjunctivePredicateToJoinRule;
-import org.apache.asterix.optimizer.rules.SetAsterixPhysicalOperatorsRule;
-import org.apache.asterix.optimizer.rules.util.EquivalenceClassUtils;
 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.IRequestParameters;
+import org.apache.asterix.translator.ResultMetadata;
 import org.apache.asterix.translator.SessionConfig;
 import org.apache.asterix.translator.SessionOutput;
-import org.apache.asterix.translator.SqlppExpressionToPlanTranslator;
 import org.apache.asterix.utils.ResourceUtils;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -106,6 +97,8 @@
 import org.apache.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.data.IPrinterFactoryProvider;
+import org.apache.hyracks.algebricks.runtime.serializer.ResultSerializerFactoryProvider;
+import org.apache.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
 import org.apache.hyracks.api.client.IClusterInfoCollector;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.client.NodeControllerInfo;
@@ -121,7 +114,6 @@
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectWriter;
-import com.google.common.collect.ImmutableSet;
 
 /**
  * Provides helper methods for compilation of a query into a JobSpec and submission
@@ -133,29 +125,11 @@
 
     public static final String PREFIX_INTERNAL_PARAMETERS = "_internal";
 
-    // A white list of supported configurable parameters.
-    private static final Set<String> CONFIGURABLE_PARAMETER_NAMES = ImmutableSet.of(
-            CompilerProperties.COMPILER_JOINMEMORY_KEY, CompilerProperties.COMPILER_GROUPMEMORY_KEY,
-            CompilerProperties.COMPILER_SORTMEMORY_KEY, CompilerProperties.COMPILER_WINDOWMEMORY_KEY,
-            CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY, CompilerProperties.COMPILER_PARALLELISM_KEY,
-            CompilerProperties.COMPILER_SORT_PARALLEL_KEY, CompilerProperties.COMPILER_SORT_SAMPLES_KEY,
-            CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY, CompilerProperties.COMPILER_INDEXONLY_KEY,
-            CompilerProperties.COMPILER_INTERNAL_SANITYCHECK_KEY,
-            CompilerProperties.COMPILER_EXTERNAL_FIELD_PUSHDOWN_KEY, CompilerProperties.COMPILER_SUBPLAN_MERGE_KEY,
-            CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY, CompilerProperties.COMPILER_ARRAYINDEX_KEY,
-            CompilerProperties.COMPILER_MIN_MEMORY_ALLOCATION_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
-            FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
-            StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
-            FeedActivityDetails.COLLECT_LOCATIONS, SqlppQueryRewriter.INLINE_WITH_OPTION,
-            SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION, "hash_merge", "output-record-type",
-            DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION,
-            SetAsterixPhysicalOperatorsRule.REWRITE_ATTEMPT_BATCH_ASSIGN,
-            EquivalenceClassUtils.REWRITE_INTERNAL_QUERYUID_PK, SqlppQueryRewriter.SQL_COMPAT_OPTION);
-
     private final IRewriterFactory rewriterFactory;
     private final IAstPrintVisitorFactory astPrintVisitorFactory;
     private final ILangExpressionToPlanTranslatorFactory translatorFactory;
     private final IRuleSetFactory ruleSetFactory;
+    private final Set<String> configurableParameterNames;
     private final ExecutionPlans executionPlans;
 
     public APIFramework(ILangCompilationProvider compilationProvider) {
@@ -163,6 +137,7 @@
         this.astPrintVisitorFactory = compilationProvider.getAstPrintVisitorFactory();
         this.translatorFactory = compilationProvider.getExpressionToPlanTranslatorFactory();
         this.ruleSetFactory = compilationProvider.getRuleSetFactory();
+        this.configurableParameterNames = compilationProvider.getCompilerOptions();
         executionPlans = new ExecutionPlans();
     }
 
@@ -181,17 +156,21 @@
                 IConflictingTypeResolver conflictingTypeResolver, PhysicalOptimizationConfig physicalOptimizationConfig,
                 AlgebricksPartitionConstraint clusterLocations, IWarningCollector warningCollector) {
             IPlanPrettyPrinter prettyPrinter = PlanPrettyPrinter.createStringPlanPrettyPrinter();
-            return new AsterixOptimizationContext(varCounter, expressionEvalSizeComputer,
+            return new AsterixOptimizationContext(this, varCounter, expressionEvalSizeComputer,
                     mergeAggregationExpressionFactory, expressionTypeComputer, missableTypeComputer,
                     conflictingTypeResolver, physicalOptimizationConfig, clusterLocations, prettyPrinter,
                     warningCollector);
         }
+
+        @Override
+        public IOptimizationContext cloneOptimizationContext(IOptimizationContext oc) {
+            return new AsterixOptimizationContext((AsterixOptimizationContext) oc);
+        }
     }
 
-    public Pair<IReturningStatement, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
-            List<ViewDecl> declaredViews, MetadataProvider metadataProvider, IReturningStatement q,
-            SessionOutput output, boolean allowNonStoredUdfCalls, boolean inlineUdfsAndViews,
-            Collection<VarIdentifier> externalVars, IWarningCollector warningCollector) throws CompilationException {
+    public Pair<IReturningStatement, Integer> reWriteQuery(LangRewritingContext langRewritingContext,
+            IReturningStatement q, SessionOutput output, boolean allowNonStoredUdfCalls, boolean inlineUdfsAndViews,
+            Collection<VarIdentifier> externalVars) throws CompilationException {
         if (q == null) {
             return null;
         }
@@ -200,9 +179,7 @@
             generateExpressionTree(q);
         }
         IQueryRewriter rw = rewriterFactory.createQueryRewriter();
-        LangRewritingContext rwCtx = new LangRewritingContext(metadataProvider, declaredFunctions, declaredViews,
-                warningCollector, q.getVarCounter());
-        rw.rewrite(rwCtx, q, allowNonStoredUdfCalls, inlineUdfsAndViews, externalVars);
+        rw.rewrite(langRewritingContext, q, allowNonStoredUdfCalls, inlineUdfsAndViews, externalVars);
         return new Pair<>(q, q.getVarCounter());
     }
 
@@ -237,7 +214,8 @@
                 && conf.is(SessionConfig.OOB_LOGICAL_PLAN)) {
             generateLogicalPlan(plan, output.config().getPlanFormat());
         }
-        CompilerProperties compilerProperties = metadataProvider.getApplicationContext().getCompilerProperties();
+        ICcApplicationContext ccAppContext = metadataProvider.getApplicationContext();
+        CompilerProperties compilerProperties = ccAppContext.getCompilerProperties();
         Map<String, Object> querySpecificConfig = validateConfig(metadataProvider.getConfig(), sourceLoc);
         final PhysicalOptimizationConfig physOptConf =
                 OptimizationConfUtil.createPhysicalOptimizationConf(compilerProperties, querySpecificConfig, sourceLoc);
@@ -245,8 +223,9 @@
         HeuristicCompilerFactoryBuilder builder =
                 new HeuristicCompilerFactoryBuilder(OptimizationContextFactory.INSTANCE);
         builder.setPhysicalOptimizationConfig(physOptConf);
-        builder.setLogicalRewrites(ruleSetFactory.getLogicalRewrites(metadataProvider.getApplicationContext()));
-        builder.setPhysicalRewrites(ruleSetFactory.getPhysicalRewrites(metadataProvider.getApplicationContext()));
+        builder.setLogicalRewrites(() -> ruleSetFactory.getLogicalRewrites(ccAppContext));
+        builder.setLogicalRewritesByKind(kind -> ruleSetFactory.getLogicalRewrites(kind, ccAppContext));
+        builder.setPhysicalRewrites(() -> ruleSetFactory.getPhysicalRewrites(ccAppContext));
         IDataFormat format = metadataProvider.getDataFormat();
         ICompilerFactory compilerFactory = builder.create();
         builder.setExpressionEvalSizeComputer(format.getExpressionEvalSizeComputer());
@@ -264,6 +243,24 @@
                 chooseLocations(clusterInfoCollector, parallelism, metadataProvider.getClusterLocations());
         builder.setClusterLocations(computationLocations);
 
+        builder.setBinaryBooleanInspectorFactory(format.getBinaryBooleanInspectorFactory());
+        builder.setBinaryIntegerInspectorFactory(format.getBinaryIntegerInspectorFactory());
+        builder.setComparatorFactoryProvider(format.getBinaryComparatorFactoryProvider());
+        builder.setExpressionRuntimeProvider(
+                new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())));
+        builder.setHashFunctionFactoryProvider(format.getBinaryHashFunctionFactoryProvider());
+        builder.setHashFunctionFamilyProvider(format.getBinaryHashFunctionFamilyProvider());
+        builder.setMissingWriterFactory(format.getMissingWriterFactory());
+        builder.setNullWriterFactory(format.getNullWriterFactory());
+        builder.setUnnestingPositionWriterFactory(format.getUnnestingPositionWriterFactory());
+        builder.setPredicateEvaluatorFactoryProvider(format.getPredicateEvaluatorFactoryProvider());
+        builder.setPrinterProvider(getPrinterFactoryProvider(format, conf.fmt()));
+        builder.setWriterFactory(PrinterBasedWriterFactory.INSTANCE);
+        builder.setResultSerializerFactoryProvider(ResultSerializerFactoryProvider.INSTANCE);
+        builder.setSerializerDeserializerProvider(format.getSerdeProvider());
+        builder.setTypeTraitProvider(format.getTypeTraitProvider());
+        builder.setNormalizedKeyComputerFactoryProvider(format.getNormalizedKeyComputerFactoryProvider());
+
         ICompiler compiler = compilerFactory.createCompiler(plan, metadataProvider, t.getVarCounter());
         if (conf.isOptimize()) {
             compiler.optimize();
@@ -273,10 +270,6 @@
                     AlgebricksStringBuilderWriter buf = new AlgebricksStringBuilderWriter(PlanPrettyPrinter.INIT_SIZE);
                     PlanPrettyPrinter.printPhysicalOps(plan, buf, 0, true);
                     output.out().write(buf.toString());
-                } else {
-                    if (isQuery || isLoad) {
-                        generateOptimizedLogicalPlan(plan, output.config().getPlanFormat());
-                    }
                 }
             }
         }
@@ -296,6 +289,41 @@
 
         boolean printSignature = isQuery && requestParameters != null && requestParameters.isPrintSignature();
 
+        if (printSignature && !isExplainOnly) { //explainOnly adds the signature later
+            printer.addResultPrinter(SignaturePrinter.newInstance(executionPlans));
+        }
+
+        if (!conf.isGenerateJobSpec()) {
+            if (isQuery || isLoad) {
+                generateOptimizedLogicalPlan(plan, output.config().getPlanFormat());
+            }
+            return null;
+        }
+
+        JobEventListenerFactory jobEventListenerFactory =
+                new JobEventListenerFactory(txnId, metadataProvider.isWriteTransaction());
+        JobSpecification spec = compiler.createJob(ccAppContext, jobEventListenerFactory);
+
+        if (isQuery) {
+            if (requestParameters == null || !requestParameters.isSkipAdmissionPolicy()) {
+                // Sets a required capacity, only for read-only queries.
+                // DDLs and DMLs are considered not that frequent.
+                // limit the computation locations to the locations that will be used in the query
+                final INodeJobTracker nodeJobTracker = ccAppContext.getNodeJobTracker();
+                final AlgebricksAbsolutePartitionConstraint jobLocations =
+                        getJobLocations(spec, nodeJobTracker, computationLocations);
+                final IClusterCapacity jobRequiredCapacity =
+                        ResourceUtils.getRequiredCapacity(plan, jobLocations, physOptConf);
+                spec.setRequiredClusterCapacity(jobRequiredCapacity);
+            }
+        }
+
+        if (conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN) || isExplainOnly) {
+            if (isQuery || isLoad) {
+                generateOptimizedLogicalPlan(plan, spec.getLogical2PhysicalMap(), output.config().getPlanFormat());
+            }
+        }
+
         if (isExplainOnly) {
             printPlanAsResult(metadataProvider, output, printer, printSignature);
             if (!conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN)) {
@@ -304,47 +332,6 @@
             return null;
         }
 
-        if (printSignature) {
-            printer.addResultPrinter(SignaturePrinter.newInstance(executionPlans));
-        }
-
-        if (!conf.isGenerateJobSpec()) {
-            return null;
-        }
-
-        builder.setBinaryBooleanInspectorFactory(format.getBinaryBooleanInspectorFactory());
-        builder.setBinaryIntegerInspectorFactory(format.getBinaryIntegerInspectorFactory());
-        builder.setComparatorFactoryProvider(format.getBinaryComparatorFactoryProvider());
-        builder.setExpressionRuntimeProvider(
-                new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())));
-        builder.setHashFunctionFactoryProvider(format.getBinaryHashFunctionFactoryProvider());
-        builder.setHashFunctionFamilyProvider(format.getBinaryHashFunctionFamilyProvider());
-        builder.setMissingWriterFactory(format.getMissingWriterFactory());
-        builder.setNullWriterFactory(format.getNullWriterFactory());
-        builder.setUnnestingPositionWriterFactory(format.getUnnestingPositionWriterFactory());
-        builder.setPredicateEvaluatorFactoryProvider(format.getPredicateEvaluatorFactoryProvider());
-        builder.setPrinterProvider(getPrinterFactoryProvider(format, conf.fmt()));
-        builder.setSerializerDeserializerProvider(format.getSerdeProvider());
-        builder.setTypeTraitProvider(format.getTypeTraitProvider());
-        builder.setNormalizedKeyComputerFactoryProvider(format.getNormalizedKeyComputerFactoryProvider());
-
-        JobEventListenerFactory jobEventListenerFactory =
-                new JobEventListenerFactory(txnId, metadataProvider.isWriteTransaction());
-        JobSpecification spec = compiler.createJob(metadataProvider.getApplicationContext(), jobEventListenerFactory);
-
-        if (isQuery) {
-            if (requestParameters == null || !requestParameters.isSkipAdmissionPolicy()) {
-                // Sets a required capacity, only for read-only queries.
-                // DDLs and DMLs are considered not that frequent.
-                // limit the computation locations to the locations that will be used in the query
-                final INodeJobTracker nodeJobTracker = metadataProvider.getApplicationContext().getNodeJobTracker();
-                final AlgebricksAbsolutePartitionConstraint jobLocations =
-                        getJobLocations(spec, nodeJobTracker, computationLocations);
-                final IClusterCapacity jobRequiredCapacity =
-                        ResourceUtils.getRequiredCapacity(plan, jobLocations, physOptConf);
-                spec.setRequiredClusterCapacity(jobRequiredCapacity);
-            }
-        }
         if (isQuery && conf.is(SessionConfig.OOB_HYRACKS_JOB)) {
             generateJob(spec);
         }
@@ -502,10 +489,10 @@
     }
 
     // Validates if the query contains unsupported query parameters.
-    private static Map<String, Object> validateConfig(Map<String, Object> config, SourceLocation sourceLoc)
+    private Map<String, Object> validateConfig(Map<String, Object> config, SourceLocation sourceLoc)
             throws AlgebricksException {
         for (String parameterName : config.keySet()) {
-            if (!CONFIGURABLE_PARAMETER_NAMES.contains(parameterName)
+            if (!configurableParameterNames.contains(parameterName)
                     && !parameterName.startsWith(PREFIX_INTERNAL_PARAMETERS)) {
                 throw AsterixException.create(ErrorCode.COMPILATION_UNSUPPORTED_QUERY_PARAMETER, sourceLoc,
                         parameterName);
@@ -534,6 +521,11 @@
         executionPlans.setLogicalPlan(getPrettyPrintVisitor(format).printPlan(plan).toString());
     }
 
+    private void generateOptimizedLogicalPlan(ILogicalPlan plan, Map<Object, String> log2phys,
+            SessionConfig.PlanFormat format) throws AlgebricksException {
+        executionPlans.setOptimizedLogicalPlan(getPrettyPrintVisitor(format).printPlan(plan, log2phys).toString());
+    }
+
     private void generateOptimizedLogicalPlan(ILogicalPlan plan, SessionConfig.PlanFormat format)
             throws AlgebricksException {
         executionPlans.setOptimizedLogicalPlan(getPrettyPrintVisitor(format).printPlan(plan).toString());
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 4dadf55..bf77c24 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
@@ -28,8 +28,6 @@
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.result.IResultSet;
 import org.apache.hyracks.http.server.AbstractServlet;
-import org.apache.hyracks.ipc.exceptions.IPCException;
-import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -61,13 +59,7 @@
     }
 
     protected IResultSet getResultSet() throws Exception { // NOSONAR
-        try {
-            return ServletUtil.getResultSet(getHyracksClientConnection(), appCtx, ctx);
-        } catch (IPCException e) {
-            LOGGER.log(Level.WARN, "Failed getting hyracks dataset connection. Resetting hyracks connection.", e);
-            ctx.put(HYRACKS_CONNECTION_ATTR, appCtx.getHcc());
-            return ServletUtil.getResultSet(getHyracksClientConnection(), appCtx, ctx);
-        }
+        return ServletUtil.getResultSet(appCtx, ctx);
     }
 
     protected IHyracksClientConnection getHyracksClientConnection() throws Exception { // NOSONAR
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
index 081c69a..56ad88e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
@@ -134,7 +134,7 @@
         try {
             // TODO: warnings should be retrieved from warnings collectors
             IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
-            IResultSet resultSet = ServletUtil.getResultSet(hcc, appCtx, ctx);
+            IResultSet resultSet = ServletUtil.getResultSet(appCtx, ctx);
             IParser parser = parserFactory.createParser(query);
             List<Statement> statements = parser.parse();
             SessionConfig sessionConfig = new SessionConfig(format, true, isSet(executeQuery), true, planFormat);
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 1ec7405..d2d7a9b 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
@@ -204,6 +204,7 @@
         stats.setJobProfile(responseStats.getJobProfile());
         stats.setProcessedObjects(responseStats.getProcessedObjects());
         stats.updateTotalWarningsCount(responseStats.getTotalWarningsCount());
+        stats.setCompileTime(responseStats.getCompileTime());
     }
 
     private static void updatePropertiesFromCC(IStatementExecutor.StatementProperties statementProperties,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
index 981cdc9..d9df051 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
@@ -21,7 +21,6 @@
 import java.io.IOException;
 import java.util.concurrent.ConcurrentMap;
 
-import org.apache.asterix.api.common.ResultMetadata;
 import org.apache.asterix.app.result.ResponseMetrics;
 import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.app.result.ResultHandle;
@@ -31,6 +30,7 @@
 import org.apache.asterix.app.result.fields.ResultsPrinter;
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
+import org.apache.asterix.translator.ResultMetadata;
 import org.apache.asterix.translator.SessionConfig;
 import org.apache.asterix.translator.SessionOutput;
 import org.apache.hyracks.api.exceptions.ErrorCode;
@@ -102,7 +102,7 @@
                 printer.printResults();
                 ResponseMetrics metrics = ResponseMetrics.of(System.nanoTime() - elapsedStart,
                         metadata.getJobDuration(), stats.getCount(), stats.getSize(), metadata.getProcessedObjects(), 0,
-                        metadata.getTotalWarningsCount());
+                        metadata.getTotalWarningsCount(), stats.getCompileTime());
                 printer.addFooterPrinter(new MetricsPrinter(metrics, HttpUtil.getPreferredCharset(request)));
                 if (metadata.getJobProfile() != null) {
                     printer.addFooterPrinter(new ProfilePrinter(metadata.getJobProfile()));
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 9f83aa8..e14c185 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
@@ -357,9 +357,9 @@
             // in case of ASYNC delivery, the status is printed by query translator
             responsePrinter.addFooterPrinter(new StatusPrinter(executionState.getResultStatus()));
         }
-        final ResponseMetrics metrics =
-                ResponseMetrics.of(System.nanoTime() - elapsedStart, executionState.duration(), stats.getCount(),
-                        stats.getSize(), stats.getProcessedObjects(), errorCount, stats.getTotalWarningsCount());
+        final ResponseMetrics metrics = ResponseMetrics.of(System.nanoTime() - elapsedStart, executionState.duration(),
+                stats.getCount(), stats.getSize(), stats.getProcessedObjects(), errorCount,
+                stats.getTotalWarningsCount(), stats.getCompileTime());
         responsePrinter.addFooterPrinter(new MetricsPrinter(metrics, resultCharset));
         if (isPrintingProfile(stats)) {
             responsePrinter.addFooterPrinter(new ProfilePrinter(stats.getJobProfile()));
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
index 3ac37bf..8d5e4db 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
@@ -24,29 +24,22 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.commons.codec.DecoderException;
 import org.apache.commons.codec.net.URLCodec;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.result.IResultSet;
-import org.apache.hyracks.client.result.ResultSet;
 import org.apache.hyracks.http.api.IServletRequest;
 
 public class ServletUtil {
-    static IResultSet getResultSet(IHyracksClientConnection hcc, IApplicationContext appCtx,
-            final Map<String, Object> ctx) throws Exception {
+    static IResultSet getResultSet(IApplicationContext appCtx, final Map<String, Object> ctx) throws Exception {
         IResultSet resultSet = (IResultSet) ctx.get(RESULTSET_ATTR);
         if (resultSet == null) {
             synchronized (ctx) {
                 resultSet = (IResultSet) ctx.get(RESULTSET_ATTR);
                 if (resultSet == null) {
-                    resultSet = new ResultSet(hcc,
-                            appCtx.getServiceContext().getControllerService().getNetworkSecurityManager()
-                                    .getSocketChannelFactory(),
-                            appCtx.getCompilerProperties().getFrameSize(), ResultReader.NUM_READERS);
+                    resultSet = appCtx.getResultSet();
                     ctx.put(RESULTSET_ATTR, resultSet);
                 }
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CcApplicationContext.java
similarity index 87%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CcApplicationContext.java
index 66f0e73..880880e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CcApplicationContext.java
@@ -16,12 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.utils;
+package org.apache.asterix.app.cc;
 
 import java.io.IOException;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.Supplier;
 
+import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.common.api.IConfigValidator;
 import org.apache.asterix.common.api.IConfigValidatorFactory;
 import org.apache.asterix.common.api.ICoordinationService;
@@ -56,13 +57,21 @@
 import org.apache.asterix.runtime.compression.CompressionManager;
 import org.apache.asterix.runtime.job.listener.NodeJobTracker;
 import org.apache.asterix.runtime.transaction.ResourceIdManager;
+import org.apache.asterix.runtime.utils.BulkTxnIdFactory;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
+import org.apache.asterix.runtime.utils.NoOpCoordinationService;
+import org.apache.asterix.runtime.utils.RequestTracker;
+import org.apache.asterix.runtime.utils.RuntimeComponentsProvider;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.ICCServiceContext;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IJobLifecycleListener;
+import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.client.result.ResultSet;
 import org.apache.hyracks.ipc.impl.HyracksConnection;
 import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.util.NetworkUtil;
 
 /*
  * Acts as an holder class for IndexRegistryProvider, AsterixStorageManager
@@ -87,7 +96,8 @@
     private MessagingProperties messagingProperties;
     private NodeProperties nodeProperties;
     private Supplier<IMetadataBootstrap> metadataBootstrapSupplier;
-    private IHyracksClientConnection hcc;
+    private volatile HyracksConnection hcc;
+    private volatile ResultSet resultSet;
     private Object extensionManager;
     private INcLifecycleCoordinator ftStrategy;
     private IJobLifecycleListener activeLifeCycleListener;
@@ -103,7 +113,7 @@
     private final IAdapterFactoryService adapterFactoryService;
     private final ReentrantReadWriteLock compilationLock = new ReentrantReadWriteLock(true);
 
-    public CcApplicationContext(ICCServiceContext ccServiceCtx, IHyracksClientConnection hcc,
+    public CcApplicationContext(ICCServiceContext ccServiceCtx, HyracksConnection hcc,
             Supplier<IMetadataBootstrap> metadataBootstrapSupplier, IGlobalRecoveryManager globalRecoveryManager,
             INcLifecycleCoordinator ftStrategy, IJobLifecycleListener activeLifeCycleListener,
             IStorageComponentProvider storageComponentProvider, IMetadataLockManager mdLockManager,
@@ -188,18 +198,44 @@
 
     @Override
     public IHyracksClientConnection getHcc() throws HyracksDataException {
-        if (!hcc.isConnected()) {
+        HyracksConnection hc = hcc;
+        if (!hc.isConnected()) {
             synchronized (this) {
-                if (!hcc.isConnected()) {
+                hc = hcc;
+                if (!hc.isConnected()) {
                     try {
-                        hcc = new HyracksConnection(hcc.getHost(), hcc.getPort());
+                        ResultSet rs = resultSet;
+                        resultSet = null;
+                        NetworkUtil.closeQuietly(rs);
+
+                        NetworkUtil.closeQuietly(hc);
+                        hcc = hc = new HyracksConnection(hcc.getHost(), hcc.getPort());
                     } catch (Exception e) {
                         throw HyracksDataException.create(e);
                     }
                 }
             }
         }
-        return hcc;
+        return hc;
+    }
+
+    @Override
+    public IResultSet getResultSet() throws HyracksDataException {
+        ResultSet rs = resultSet;
+        if (rs == null) {
+            synchronized (this) {
+                rs = resultSet;
+                if (rs == null) {
+                    try {
+                        resultSet = rs = ResultReader.createResultSet(getHcc(), ccServiceCtx.getControllerService(),
+                                compilerProperties);
+                    } catch (Exception e) {
+                        throw HyracksDataException.create(e);
+                    }
+                }
+            }
+        }
+        return rs;
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
index 30eaf93..6c0382d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.ISecondaryIndexOperationsHelper;
 import org.apache.asterix.metadata.utils.SecondaryIndexOperationsHelper;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -63,7 +64,7 @@
         if (index.isPrimaryIndex()) {
             throw new CompilationException(ErrorCode.OPERATION_NOT_SUPPORTED_ON_PRIMARY_INDEX, loc, indexName);
         }
-        SecondaryIndexOperationsHelper secondaryIndexHelper =
+        ISecondaryIndexOperationsHelper secondaryIndexHelper =
                 SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, loc);
         IndexDataflowHelperFactory indexDataflowHelperFactory =
                 new IndexDataflowHelperFactory(metadataProvider.getStorageComponentProvider().getStorageManager(),
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 1a89168..a46522e 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
@@ -27,6 +27,7 @@
 import java.util.concurrent.ExecutorService;
 
 import org.apache.asterix.active.ActiveManager;
+import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.common.api.IConfigValidator;
 import org.apache.asterix.common.api.IConfigValidatorFactory;
 import org.apache.asterix.common.api.ICoordinationService;
@@ -86,6 +87,8 @@
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponentManager;
 import org.apache.hyracks.api.network.INetworkSecurityManager;
+import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.client.result.ResultSet;
 import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.ipc.impl.HyracksConnection;
@@ -108,6 +111,7 @@
 import org.apache.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
 import org.apache.hyracks.storage.common.file.IResourceIdFactory;
 import org.apache.hyracks.util.MaintainedThreadNameExecutorService;
+import org.apache.hyracks.util.NetworkUtil;
 import org.apache.hyracks.util.cache.CacheManager;
 import org.apache.hyracks.util.cache.ICacheManager;
 import org.apache.logging.log4j.Level;
@@ -147,7 +151,8 @@
     private final NCExtensionManager ncExtensionManager;
     private final IStorageComponentProvider componentProvider;
     private final IPersistedResourceRegistry persistedResourceRegistry;
-    private IHyracksClientConnection hcc;
+    private volatile HyracksConnection hcc;
+    private volatile ResultSet resultSet;
     private IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
     private IReplicaManager replicaManager;
     private IReceptionist receptionist;
@@ -513,15 +518,22 @@
 
     @Override
     public IHyracksClientConnection getHcc() throws HyracksDataException {
-        if (hcc == null || !hcc.isConnected()) {
+        HyracksConnection hc = hcc;
+        if (hc == null || !hc.isConnected()) {
             synchronized (this) {
-                if (hcc == null || !hcc.isConnected()) {
+                hc = hcc;
+                if (hc == null || !hc.isConnected()) {
                     try {
+                        ResultSet rs = resultSet;
+                        resultSet = null;
+                        NetworkUtil.closeQuietly(rs);
+
                         NodeControllerService ncSrv = (NodeControllerService) ncServiceContext.getControllerService();
                         // TODO(mblow): multicc
                         CcId primaryCcId = ncSrv.getPrimaryCcId();
                         ClusterControllerInfo ccInfo = ncSrv.getNodeParameters(primaryCcId).getClusterControllerInfo();
-                        hcc = new HyracksConnection(ccInfo.getClientNetAddress(), ccInfo.getClientNetPort(),
+                        NetworkUtil.closeQuietly(hc);
+                        hcc = hc = new HyracksConnection(ccInfo.getClientNetAddress(), ccInfo.getClientNetPort(),
                                 ncSrv.getNetworkSecurityManager().getSocketChannelFactory());
                     } catch (Exception e) {
                         throw HyracksDataException.create(e);
@@ -529,7 +541,26 @@
                 }
             }
         }
-        return hcc;
+        return hc;
+    }
+
+    @Override
+    public IResultSet getResultSet() throws HyracksDataException {
+        ResultSet rs = resultSet;
+        if (rs == null) {
+            synchronized (this) {
+                rs = resultSet;
+                if (rs == null) {
+                    try {
+                        resultSet = rs = ResultReader.createResultSet(getHcc(), ncServiceContext.getControllerService(),
+                                compilerProperties);
+                    } catch (Exception e) {
+                        throw HyracksDataException.create(e);
+                    }
+                }
+            }
+        }
+        return rs;
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/JobResultCallback.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/JobResultCallback.java
index 05073f9..66d81d2 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/JobResultCallback.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/JobResultCallback.java
@@ -23,8 +23,8 @@
 import java.util.Iterator;
 import java.util.Set;
 
-import org.apache.asterix.api.common.ResultMetadata;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.translator.ResultMetadata;
 import org.apache.hyracks.api.exceptions.Warning;
 import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.api.job.JobId;
@@ -82,7 +82,7 @@
             for (JobletProfile jp : jobletProfiles) {
                 final Collection<TaskProfile> jobletTasksProfile = jp.getTaskProfiles().values();
                 for (TaskProfile tp : jobletTasksProfile) {
-                    processedObjects += tp.getStatsCollector().getAggregatedStats().getTupleCounter().get();
+                    processedObjects += tp.getStatsCollector().getAggregatedStats().getInputTupleCounter().get();
                     aggregateTotalWarningsCount += tp.getTotalWarningsCount();
                     Set<Warning> taskWarnings = tp.getWarnings();
                     if (AggregateWarnings.size() < maxWarnings && !taskWarnings.isEmpty()) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMetrics.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMetrics.java
index 8dfe923..d55bb07 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMetrics.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMetrics.java
@@ -28,12 +28,13 @@
     private long errorCount;
     private long warnCount;
     private long diskIoCount;
+    private long compileTime;
 
     private ResponseMetrics() {
     }
 
     public static ResponseMetrics of(long elapsedTime, long executionTime, long resultCount, long resultSize,
-            long processedObjects, long errorCount, long warnCount) {
+            long processedObjects, long errorCount, long warnCount, long compileTime) {
         ResponseMetrics metrics = new ResponseMetrics();
         metrics.elapsedTime = elapsedTime;
         metrics.executionTime = executionTime;
@@ -42,6 +43,7 @@
         metrics.processedObjects = processedObjects;
         metrics.errorCount = errorCount;
         metrics.warnCount = warnCount;
+        metrics.compileTime = compileTime;
         return metrics;
     }
 
@@ -72,4 +74,8 @@
     public long getWarnCount() {
         return warnCount;
     }
+
+    public long getCompileTime() {
+        return compileTime;
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
index 1acae87..56dfd5e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.app.result;
 
+import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.comm.IFrame;
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -27,12 +29,14 @@
 import org.apache.hyracks.api.result.IResultSetReader;
 import org.apache.hyracks.api.result.ResultJobRecord.Status;
 import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.api.service.IControllerService;
+import org.apache.hyracks.client.result.ResultSet;
 import org.apache.hyracks.dataflow.common.comm.io.ResultFrameTupleAccessor;
 
 public class ResultReader {
-    private IResultSetReader reader;
+    private final IResultSetReader reader;
 
-    private IFrameTupleAccessor frameTupleAccessor;
+    private final IFrameTupleAccessor frameTupleAccessor;
 
     // Number of parallel result reader buffers
     public static final int NUM_READERS = 1;
@@ -57,4 +61,10 @@
     public IResultMetadata getMetadata() {
         return reader.getResultMetadata();
     }
+
+    public static ResultSet createResultSet(IHyracksClientConnection hcc, IControllerService srv,
+            CompilerProperties compilerProperties) throws Exception {
+        return new ResultSet(hcc, srv.getNetworkSecurityManager().getSocketChannelFactory(),
+                compilerProperties.getFrameSize(), ResultReader.NUM_READERS);
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java
index 5549683..6b1d670 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java
@@ -32,6 +32,7 @@
     public enum Metrics {
         ELAPSED_TIME("elapsedTime"),
         EXECUTION_TIME("executionTime"),
+        COMPILE_TIME("compileTime"),
         RESULT_COUNT("resultCount"),
         RESULT_SIZE("resultSize"),
         ERROR_COUNT("errorCount"),
@@ -71,6 +72,8 @@
         ResultUtil.printField(pw, Metrics.EXECUTION_TIME.str(),
                 Duration.formatNanos(metrics.getExecutionTime(), useAscii));
         pw.print("\n\t");
+        ResultUtil.printField(pw, Metrics.COMPILE_TIME.str(), Duration.formatNanos(metrics.getCompileTime(), useAscii));
+        pw.print("\n\t");
         ResultUtil.printField(pw, Metrics.RESULT_COUNT.str(), metrics.getResultCount(), true);
         pw.print("\n\t");
         ResultUtil.printField(pw, Metrics.RESULT_SIZE.str(), metrics.getResultSize(), true);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
index 048584c..3f1e632 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
@@ -22,7 +22,6 @@
 import java.util.LinkedHashSet;
 import java.util.List;
 
-import org.apache.asterix.api.common.ResultMetadata;
 import org.apache.asterix.api.http.server.ResultUtil;
 import org.apache.asterix.common.annotations.IRecordTypeAnnotation;
 import org.apache.asterix.common.annotations.RecordFieldOrderAnnotation;
@@ -34,6 +33,7 @@
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.translator.ExecutionPlans;
+import org.apache.asterix.translator.ResultMetadata;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.util.JSONUtil;
 
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 1729b50..d2f0737 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
@@ -110,7 +110,10 @@
 import org.apache.asterix.lang.common.expression.TypeExpression;
 import org.apache.asterix.lang.common.expression.TypeReferenceExpression;
 import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.AdapterDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeStatement;
 import org.apache.asterix.lang.common.statement.CompactStatement;
 import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
@@ -195,6 +198,7 @@
 import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
 import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.asterix.metadata.utils.MetadataUtil;
+import org.apache.asterix.metadata.utils.SampleOperationsHelper;
 import org.apache.asterix.metadata.utils.TypeUtil;
 import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.base.IAObject;
@@ -208,6 +212,7 @@
 import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
 import org.apache.asterix.runtime.fulltext.IFullTextFilterDescriptor;
 import org.apache.asterix.runtime.fulltext.StopwordsFullTextFilterDescriptor;
+import org.apache.asterix.runtime.operators.StreamStats;
 import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
 import org.apache.asterix.translator.AbstractLangTranslator;
 import org.apache.asterix.translator.ClientRequest;
@@ -235,13 +240,11 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.core.algebra.base.Counter;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.data.IAWriterFactory;
-import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
-import org.apache.hyracks.algebricks.runtime.serializer.ResultSerializerFactoryProvider;
-import org.apache.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
 import org.apache.hyracks.api.client.IClusterInfoCollector;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -253,6 +256,7 @@
 import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 import org.apache.hyracks.api.result.IResultSet;
 import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
@@ -330,10 +334,8 @@
     public void compileAndExecute(IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
         validateStatements(requestParameters);
         trackRequest(requestParameters);
-        int resultSetIdCounter = 0;
+        Counter resultSetIdCounter = new Counter(0);
         FileSplit outputFile = null;
-        IAWriterFactory writerFactory = PrinterBasedWriterFactory.INSTANCE;
-        IResultSerializerFactoryProvider resultSerializerFactoryProvider = ResultSerializerFactoryProvider.INSTANCE;
         String threadName = Thread.currentThread().getName();
         Thread.currentThread().setName(
                 QueryTranslator.class.getSimpleName() + ":" + requestParameters.getRequestReference().getUuid());
@@ -353,8 +355,8 @@
                 }
                 validateOperation(appCtx, activeDataverse, stmt);
                 MetadataProvider metadataProvider = MetadataProvider.create(appCtx, activeDataverse);
-                configureMetadataProvider(metadataProvider, config, resultSerializerFactoryProvider, writerFactory,
-                        outputFile, requestParameters, stmt);
+                configureMetadataProvider(metadataProvider, config, resultSetIdCounter, outputFile, requestParameters,
+                        stmt);
                 IStatementRewriter stmtRewriter = rewriterFactory.createStatementRewriter();
                 rewriteStatement(stmt, stmtRewriter, metadataProvider); // Rewrite the statement's AST.
                 Statement.Kind kind = stmt.getKind();
@@ -439,16 +441,22 @@
                         handleViewDropStatement(metadataProvider, stmt);
                         break;
                     case LOAD:
+                        if (stats.getProfileType() == Stats.ProfileType.FULL) {
+                            this.jobFlags.add(JobFlag.PROFILE_RUNTIME);
+                        }
                         handleLoadStatement(metadataProvider, stmt, hcc);
                         break;
                     case INSERT:
                     case UPSERT:
                         if (((InsertStatement) stmt).getReturnExpression() != null) {
-                            metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
+                            metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter.getAndInc()));
                             metadataProvider.setResultAsyncMode(resultDelivery == ResultDelivery.ASYNC
                                     || resultDelivery == ResultDelivery.DEFERRED);
                             metadataProvider.setMaxResultReads(maxResultReads);
                         }
+                        if (stats.getProfileType() == Stats.ProfileType.FULL) {
+                            this.jobFlags.add(JobFlag.PROFILE_RUNTIME);
+                        }
                         handleInsertUpsertStatement(metadataProvider, stmt, hcc, resultSet, resultDelivery, outMetadata,
                                 stats, requestParameters, stmtParams, stmtRewriter);
                         break;
@@ -480,7 +488,7 @@
                         handleCreateFeedPolicyStatement(metadataProvider, stmt);
                         break;
                     case QUERY:
-                        metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
+                        metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter.getAndInc()));
                         metadataProvider.setResultAsyncMode(
                                 resultDelivery == ResultDelivery.ASYNC || resultDelivery == ResultDelivery.DEFERRED);
                         metadataProvider.setMaxResultReads(maxResultReads);
@@ -490,6 +498,12 @@
                         handleQuery(metadataProvider, (Query) stmt, hcc, resultSet, resultDelivery, outMetadata, stats,
                                 requestParameters, stmtParams, stmtRewriter);
                         break;
+                    case ANALYZE:
+                        handleAnalyzeStatement(metadataProvider, stmt, hcc, requestParameters);
+                        break;
+                    case ANALYZE_DROP:
+                        handleAnalyzeDropStatement(metadataProvider, stmt, hcc, requestParameters);
+                        break;
                     case COMPACT:
                         handleCompactStatement(metadataProvider, stmt, hcc);
                         break;
@@ -505,7 +519,10 @@
                     case EXTENSION:
                         final ExtensionStatement extStmt = (ExtensionStatement) stmt;
                         statementProperties.setName(extStmt.getName());
-                        extStmt.handle(hcc, this, requestParameters, metadataProvider, resultSetIdCounter);
+                        if (!isCompileOnly()) {
+                            extStmt.handle(hcc, this, requestParameters, metadataProvider,
+                                    resultSetIdCounter.getAndInc());
+                        }
                         break;
                     default:
                         throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, stmt.getSourceLocation(),
@@ -522,14 +539,13 @@
     }
 
     protected void configureMetadataProvider(MetadataProvider metadataProvider, Map<String, String> config,
-            IResultSerializerFactoryProvider resultSerializerFactoryProvider, IAWriterFactory writerFactory,
-            FileSplit outputFile, IRequestParameters requestParameters, Statement statement) {
+            Counter resultSetIdCounter, FileSplit outputFile, IRequestParameters requestParameters,
+            Statement statement) {
         if (statement.getKind() == Statement.Kind.QUERY && requestParameters.isSQLCompatMode()) {
             metadataProvider.getConfig().put(SqlppQueryRewriter.SQL_COMPAT_OPTION, Boolean.TRUE.toString());
         }
         metadataProvider.getConfig().putAll(config);
-        metadataProvider.setWriterFactory(writerFactory);
-        metadataProvider.setResultSerializerFactoryProvider(resultSerializerFactoryProvider);
+        metadataProvider.setResultSetIdCounter(resultSetIdCounter);
         metadataProvider.setOutputFile(outputFile);
     }
 
@@ -552,6 +568,13 @@
         config.put(pname, pvalue);
     }
 
+    protected LangRewritingContext createLangRewritingContext(MetadataProvider metadataProvider,
+            List<FunctionDecl> declaredFunctions, List<ViewDecl> declaredViews, IWarningCollector warningCollector,
+            int varCounter) {
+        return new LangRewritingContext(metadataProvider, declaredFunctions, declaredViews, warningCollector,
+                varCounter);
+    }
+
     protected Pair<IAWriterFactory, FileSplit> handleWriteStatement(Statement stmt)
             throws InstantiationException, IllegalAccessException, ClassNotFoundException {
         WriteStatement ws = (WriteStatement) stmt;
@@ -610,6 +633,9 @@
         CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
         DataverseName dvName = stmtCreateDataverse.getDataverseName();
         metadataProvider.validateDataverseName(dvName, stmtCreateDataverse.getSourceLocation());
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createDataverseBegin(lockManager, metadataProvider.getLocks(), dvName);
         try {
             doCreateDataverseStatement(metadataProvider, stmtCreateDataverse, requestParameters);
@@ -713,6 +739,9 @@
         String compactionPolicy = dd.getCompactionPolicy();
         boolean defaultCompactionPolicy = compactionPolicy == null;
 
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName,
                 itemTypeDataverseName, itemTypeName, itemTypeAnonymous, metaItemTypeDataverseName, metaItemTypeName,
                 metaItemTypeAnonymous, nodegroupName, compactionPolicy, defaultCompactionPolicy, dd.getDatasetType(),
@@ -1061,6 +1090,9 @@
         metadataProvider.validateDatabaseObjectName(stmtCreateIndex.getDataverseName(), indexName,
                 stmt.getSourceLocation());
         DataverseName dataverseName = getActiveDataverseName(stmtCreateIndex.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName,
                 fullTextConfigName);
         try {
@@ -1409,6 +1441,9 @@
                 stmt.getSourceLocation());
         DataverseName dataverseName = getActiveDataverseName(stmtCreateFilter.getDataverseName());
 
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createFullTextFilterBegin(lockManager, metadataProvider.getLocks(), dataverseName, fullTextFilterName);
         try {
             doCreateFullTextFilter(metadataProvider, stmtCreateFilter, dataverseName);
@@ -1477,6 +1512,9 @@
         DataverseName dataverseName = getActiveDataverseName(stmtCreateConfig.getDataverseName());
         ImmutableList<String> filterNames = stmtCreateConfig.getFilterNames();
 
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createFullTextConfigBegin(lockManager, metadataProvider.getLocks(), dataverseName, configName,
                 filterNames);
         try {
@@ -1842,6 +1880,9 @@
         String typeName = stmtCreateType.getIdent().getValue();
         metadataProvider.validateDatabaseObjectName(stmtCreateType.getDataverseName(), typeName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(stmtCreateType.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockUtil.createTypeBegin(lockManager, metadataProvider.getLocks(), dataverseName, typeName);
@@ -1895,6 +1936,9 @@
             throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                     dataverseName + " " + dataverse() + " can't be dropped");
         }
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropDataverseBegin(lockManager, metadataProvider.getLocks(), dataverseName);
         try {
             doDropDataverse(stmtDropDataverse, metadataProvider, hcc, requestParameters);
@@ -2098,6 +2142,9 @@
         String datasetName = stmtDelete.getDatasetName().getValue();
         metadataProvider.validateDatabaseObjectName(stmtDelete.getDataverseName(), datasetName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(stmtDelete.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
         try {
             doDropDataset(dataverseName, datasetName, metadataProvider, stmtDelete.getIfExists(), hcc,
@@ -2205,6 +2252,9 @@
                 stmtIndexDrop.getIndexName().getValue(), stmtIndexDrop.getSourceLocation());
         DataverseName dataverseName = getActiveDataverseName(stmtIndexDrop.getDataverseName());
         String datasetName = stmtIndexDrop.getDatasetName().getValue();
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
         try {
             doDropIndex(metadataProvider, stmtIndexDrop, dataverseName, datasetName, hcc, requestParameters);
@@ -2245,14 +2295,8 @@
                 }
                 ensureNonPrimaryIndexDrop(index, sourceLoc);
                 validateDatasetState(metadataProvider, ds, sourceLoc);
-                // #. prepare a job to drop the index in NC.
-                jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds, sourceLoc));
-
-                // #. mark PendingDropOp on the existing index
-                MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
-                MetadataManager.INSTANCE.addIndex(mdTxnCtx,
-                        new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getIndexDetails(),
-                                index.isEnforced(), index.isPrimaryIndex(), MetadataUtil.PENDING_DROP_OP));
+                prepareIndexDrop(metadataProvider, dataverseName, datasetName, sourceLoc, indexName, jobsToExecute,
+                        mdTxnCtx, ds, index);
 
                 // #. commit the existing transaction before calling runJob.
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -2286,8 +2330,9 @@
                             "Dropping " + dataset() + " files index is not allowed.");
                 }
                 ensureNonPrimaryIndexDrop(index, sourceLoc);
-                // #. prepare a job to drop the index in NC.
-                jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds, sourceLoc));
+                prepareIndexDrop(metadataProvider, dataverseName, datasetName, sourceLoc, indexName, jobsToExecute,
+                        mdTxnCtx, ds, index);
+
                 List<Index> datasetIndexes =
                         MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
                 if (datasetIndexes.size() == 2) {
@@ -2309,12 +2354,6 @@
                     }
                 }
 
-                // #. mark PendingDropOp on the existing index
-                MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
-                MetadataManager.INSTANCE.addIndex(mdTxnCtx,
-                        new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getIndexDetails(),
-                                index.isEnforced(), index.isPrimaryIndex(), MetadataUtil.PENDING_DROP_OP));
-
                 // #. commit the existing transaction before calling runJob.
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                 bActiveTxn = false;
@@ -2387,6 +2426,9 @@
         DataverseName dataverseName = getActiveDataverseName(stmtFilterDrop.getDataverseName());
         String fullTextFilterName = stmtFilterDrop.getFilterName();
 
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropFullTextFilterBegin(lockManager, metadataProvider.getLocks(), dataverseName, fullTextFilterName);
         try {
             doDropFullTextFilter(metadataProvider, stmtFilterDrop, dataverseName, fullTextFilterName);
@@ -2427,6 +2469,9 @@
         DataverseName dataverseName = getActiveDataverseName(stmtConfigDrop.getDataverseName());
         String configName = stmtConfigDrop.getConfigName();
 
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropFullTextConfigBegin(lockManager, metadataProvider.getLocks(), dataverseName, configName);
         try {
             doDropFullTextConfig(metadataProvider, stmtConfigDrop, hcc, requestParameters);
@@ -2476,6 +2521,9 @@
         String typeName = stmtTypeDrop.getTypeName().getValue();
         metadataProvider.validateDatabaseObjectName(stmtTypeDrop.getDataverseName(), typeName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(stmtTypeDrop.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockUtil.dropTypeBegin(lockManager, metadataProvider.getLocks(), dataverseName, typeName);
@@ -2516,6 +2564,9 @@
         SourceLocation sourceLoc = stmtDelete.getSourceLocation();
         String nodegroupName = stmtDelete.getNodeGroupName().getValue();
         metadataProvider.validateDatabaseObjectName(null, nodegroupName, sourceLoc);
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockManager.acquireNodeGroupWriteLock(metadataProvider.getLocks(), nodegroupName);
@@ -2560,6 +2611,9 @@
             viewItemTypeAnonymous = false;
         }
 
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, viewName,
                 viewItemTypeDataverseName, viewItemTypeName, viewItemTypeAnonymous, null, null, false, null, null, true,
                 DatasetType.VIEW, null);
@@ -2714,8 +2768,10 @@
             IQueryRewriter queryRewriter = rewriterFactory.createQueryRewriter();
             Query wrappedQuery = queryRewriter.createViewAccessorQuery(viewDecl);
             metadataProvider.setDefaultDataverse(dv);
-            apiFramework.reWriteQuery(declaredFunctions, Collections.singletonList(viewDecl), metadataProvider,
-                    wrappedQuery, sessionOutput, false, false, Collections.emptyList(), warningCollector);
+            LangRewritingContext langRewritingContext = createLangRewritingContext(metadataProvider, declaredFunctions,
+                    Collections.singletonList(viewDecl), warningCollector, wrappedQuery.getVarCounter());
+            apiFramework.reWriteQuery(langRewritingContext, wrappedQuery, sessionOutput, false, false,
+                    Collections.emptyList());
 
             List<List<Triple<DataverseName, String, String>>> dependencies =
                     ViewUtil.getViewDependencies(viewDecl, foreignKeys, queryRewriter);
@@ -2751,6 +2807,9 @@
         String viewName = stmtDrop.getViewName().getValue();
         metadataProvider.validateDatabaseObjectName(stmtDrop.getDataverseName(), viewName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(stmtDrop.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, viewName);
         try {
             doDropView(metadataProvider, stmtDrop, dataverseName, viewName);
@@ -2833,6 +2892,9 @@
             }
         }
 
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createFunctionBegin(lockManager, metadataProvider.getLocks(), dataverseName, signature.getName(),
                 libraryDataverseName, libraryName);
         try {
@@ -2983,8 +3045,10 @@
                 fdList.addAll(declaredFunctions);
                 fdList.add(fd);
                 metadataProvider.setDefaultDataverse(dv);
-                apiFramework.reWriteQuery(fdList, null, metadataProvider, wrappedQuery, sessionOutput, false, false,
-                        Collections.emptyList(), warningCollector);
+                LangRewritingContext langRewritingContext = createLangRewritingContext(metadataProvider, fdList, null,
+                        warningCollector, wrappedQuery.getVarCounter());
+                apiFramework.reWriteQuery(langRewritingContext, wrappedQuery, sessionOutput, false, false,
+                        Collections.emptyList());
 
                 List<List<Triple<DataverseName, String, String>>> dependencies =
                         FunctionUtil.getFunctionDependencies(fd, queryRewriter);
@@ -3082,6 +3146,9 @@
                 stmtDropFunction.getSourceLocation());
         DataverseName dataverseName = getActiveDataverseName(signature.getDataverseName());
         signature.setDataverseName(dataverseName);
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropFunctionBegin(lockManager, metadataProvider.getLocks(), dataverseName, signature.getName());
         try {
             doDropFunction(metadataProvider, stmtDropFunction, signature, requestParameters);
@@ -3140,6 +3207,9 @@
             libraryDataverseName = dataverseName;
         }
         String libraryName = cas.getLibraryName();
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createAdapterBegin(lockManager, metadataProvider.getLocks(), dataverseName, adapterName,
                 libraryDataverseName, libraryName);
         try {
@@ -3210,6 +3280,9 @@
         String adapterName = stmtDropAdapter.getAdapterName();
         metadataProvider.validateDatabaseObjectName(stmtDropAdapter.getDataverseName(), adapterName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(stmtDropAdapter.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropAdapterBegin(lockManager, metadataProvider.getLocks(), dataverseName, adapterName);
         try {
             doDropAdapter(metadataProvider, stmtDropAdapter, dataverseName, adapterName);
@@ -3260,6 +3333,9 @@
         DataverseName dataverseName = getActiveDataverseName(cls.getDataverseName());
         String libraryName = cls.getLibraryName();
         String libraryHash = cls.getHash();
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createLibraryBegin(lockManager, metadataProvider.getLocks(), dataverseName, libraryName);
         try {
             doCreateLibrary(metadataProvider, dataverseName, libraryName, libraryHash, cls, hcc, requestParameters);
@@ -3388,6 +3464,9 @@
         metadataProvider.validateDatabaseObjectName(stmtDropLibrary.getDataverseName(), libraryName,
                 stmtDropLibrary.getSourceLocation());
         DataverseName dataverseName = getActiveDataverseName(stmtDropLibrary.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropLibraryBegin(lockManager, metadataProvider.getLocks(), dataverseName, libraryName);
         try {
             doDropLibrary(metadataProvider, stmtDropLibrary, dataverseName, libraryName, hcc, requestParameters);
@@ -3482,6 +3561,9 @@
         DataverseName objectDataverseName =
                 css.getObjectDataverseName() != null ? css.getObjectDataverseName() : dataverseName;
         String objectName = css.getObjectName();
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createSynonymBegin(lockManager, metadataProvider.getLocks(), dataverseName, synonymName);
         try {
             doCreateSynonym(metadataProvider, css, dataverseName, synonymName, objectDataverseName, objectName);
@@ -3529,6 +3611,9 @@
         metadataProvider.validateDatabaseObjectName(stmtSynDrop.getDataverseName(), synonymName,
                 stmtSynDrop.getSourceLocation());
         DataverseName dataverseName = getActiveDataverseName(stmtSynDrop.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropSynonymBegin(lockManager, metadataProvider.getLocks(), dataverseName, synonymName);
         try {
             doDropSynonym(metadataProvider, stmtSynDrop, dataverseName, synonymName);
@@ -3582,7 +3667,7 @@
             afterCompile();
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
-            if (spec != null && sessionConfig.isExecuteQuery()) {
+            if (spec != null && !isCompileOnly()) {
                 runJob(hcc, spec);
             }
         } catch (Exception e) {
@@ -3627,7 +3712,7 @@
                         rewriteCompileInsertUpsert(hcc, metadataProvider, stmtInsertUpsert, stmtParams);
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                 bActiveTxn = false;
-                return !sessionConfig.isExecuteQuery() ? null : jobSpec;
+                return isCompileOnly() ? null : jobSpec;
             } catch (Exception e) {
                 if (bActiveTxn) {
                     abort(e, e, mdTxnCtx);
@@ -3678,7 +3763,7 @@
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
 
-            if (jobSpec != null && sessionConfig.isExecuteQuery()) {
+            if (jobSpec != null && !isCompileOnly()) {
                 runJob(hcc, jobSpec);
             }
             return jobSpec;
@@ -3701,8 +3786,10 @@
         Map<VarIdentifier, IAObject> externalVars = createExternalVariables(query, stmtParams);
 
         // Query Rewriting (happens under the same ongoing metadata transaction)
-        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions, null,
-                metadataProvider, query, sessionOutput, true, true, externalVars.keySet(), warningCollector);
+        LangRewritingContext langRewritingContext = createLangRewritingContext(metadataProvider, declaredFunctions,
+                null, warningCollector, query.getVarCounter());
+        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(langRewritingContext, query,
+                sessionOutput, true, true, externalVars.keySet());
 
         // Query Compilation (happens under the same ongoing metadata transaction)
         return apiFramework.compileQuery(clusterInfoCollector, metadataProvider, (Query) rewrittenResult.first,
@@ -3710,7 +3797,7 @@
                 responsePrinter, warningCollector, requestParameters);
     }
 
-    private JobSpecification rewriteCompileInsertUpsert(IClusterInfoCollector clusterInfoCollector,
+    protected JobSpecification rewriteCompileInsertUpsert(IClusterInfoCollector clusterInfoCollector,
             MetadataProvider metadataProvider, InsertStatement insertUpsert, Map<String, IAObject> stmtParams)
             throws AlgebricksException, ACIDException {
         SourceLocation sourceLoc = insertUpsert.getSourceLocation();
@@ -3718,8 +3805,10 @@
         Map<VarIdentifier, IAObject> externalVars = createExternalVariables(insertUpsert, stmtParams);
 
         // Insert/upsert statement rewriting (happens under the same ongoing metadata transaction)
-        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions, null,
-                metadataProvider, insertUpsert, sessionOutput, true, true, externalVars.keySet(), warningCollector);
+        LangRewritingContext langRewritingContext = createLangRewritingContext(metadataProvider, declaredFunctions,
+                null, warningCollector, insertUpsert.getVarCounter());
+        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(langRewritingContext,
+                insertUpsert, sessionOutput, true, true, externalVars.keySet());
 
         InsertStatement rewrittenInsertUpsert = (InsertStatement) rewrittenResult.first;
         DataverseName dataverseName = getActiveDataverseName(rewrittenInsertUpsert.getDataverseName());
@@ -3755,6 +3844,9 @@
         String feedName = cfs.getFeedName().getValue();
         metadataProvider.validateDatabaseObjectName(cfs.getDataverseName(), feedName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(cfs.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockUtil.createFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, feedName);
@@ -3794,6 +3886,9 @@
         String policyName = cfps.getPolicyName();
         metadataProvider.validateDatabaseObjectName(null, policyName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(null);
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverseName, policyName);
         try {
             mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -3855,6 +3950,9 @@
         String feedName = stmtFeedDrop.getFeedName().getValue();
         metadataProvider.validateDatabaseObjectName(stmtFeedDrop.getDataverseName(), feedName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(stmtFeedDrop.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockUtil.dropFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, feedName);
@@ -3908,6 +4006,9 @@
         String policyName = stmtFeedPolicyDrop.getPolicyName().getValue();
         metadataProvider.validateDatabaseObjectName(stmtFeedPolicyDrop.getDataverseName(), policyName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(stmtFeedPolicyDrop.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockUtil.dropFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverseName, policyName);
@@ -3937,6 +4038,9 @@
         SourceLocation sourceLoc = sfs.getSourceLocation();
         DataverseName dataverseName = getActiveDataverseName(sfs.getDataverseName());
         String feedName = sfs.getFeedName().getValue();
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean committed = false;
         lockUtil.startFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, feedName);
@@ -3999,6 +4103,9 @@
             throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                     "Feed " + feedName + " is not started.");
         }
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.stopFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, feedName);
         try {
             listener.stop(metadataProvider);
@@ -4016,6 +4123,9 @@
         String datasetName = cfs.getDatasetName().getValue();
         String policyName = cfs.getPolicy();
         String whereClauseBody = cfs.getWhereClauseBody();
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         // TODO: Check whether we are connecting a change feed to a non-meta dataset
@@ -4071,6 +4181,9 @@
         DataverseName dataverseName = getActiveDataverseName(cfs.getDataverseName());
         String datasetName = cfs.getDatasetName().getValue();
         String feedName = cfs.getFeedName().getValue();
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockUtil.disconnectFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName, feedName);
@@ -4110,12 +4223,385 @@
         }
     }
 
+    protected void handleAnalyzeStatement(MetadataProvider metadataProvider, Statement stmt,
+            IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
+        AnalyzeStatement analyzeStatement = (AnalyzeStatement) stmt;
+        metadataProvider.validateDatabaseObjectName(analyzeStatement.getDataverseName(),
+                analyzeStatement.getDatasetName(), analyzeStatement.getSourceLocation());
+        DataverseName dataverseName = getActiveDataverseName(analyzeStatement.getDataverseName());
+        String datasetName = analyzeStatement.getDatasetName();
+        if (isCompileOnly()) {
+            return;
+        }
+        lockUtil.analyzeDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
+        try {
+            doAnalyzeDataset(metadataProvider, analyzeStatement, dataverseName, datasetName, hcc, requestParameters);
+        } finally {
+            metadataProvider.getLocks().unlock();
+        }
+    }
+
+    protected void doAnalyzeDataset(MetadataProvider metadataProvider, AnalyzeStatement stmtAnalyze,
+            DataverseName dataverseName, String datasetName, IHyracksClientConnection hcc,
+            IRequestParameters requestParameters) throws Exception {
+        SourceLocation sourceLoc = stmtAnalyze.getSourceLocation();
+        ProgressState progressNewIndexCreate = ProgressState.NO_PROGRESS;
+        ProgressState progressExistingIndexDrop = ProgressState.NO_PROGRESS;
+        Dataset ds = null;
+        Index existingIndex = null, newIndexPendingAdd = null;
+        JobSpecification existingIndexDropSpec = null;
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        boolean bActiveTxn = true;
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        try {
+            // Check if the dataverse exists
+            Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
+            if (dv == null) {
+                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
+            }
+            // Check if the dataset exists
+            ds = metadataProvider.findDataset(dataverseName, datasetName);
+            if (ds == null) {
+                throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                        dataverseName);
+            }
+            if (ds.getDatasetType() == DatasetType.INTERNAL) {
+                validateDatasetState(metadataProvider, ds, sourceLoc);
+            } else {
+                throw new CompilationException(ErrorCode.OPERATION_NOT_SUPPORTED, sourceLoc);
+            }
+
+            IndexType sampleIndexType = IndexType.SAMPLE;
+            Pair<String, String> sampleIndexNames = IndexUtil.getSampleIndexNames(datasetName);
+            String newIndexName;
+            existingIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+                    datasetName, sampleIndexNames.first);
+            if (existingIndex != null) {
+                newIndexName = sampleIndexNames.second;
+            } else {
+                existingIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
+                        dataverseName, datasetName, sampleIndexNames.second);
+                newIndexName = sampleIndexNames.first;
+            }
+
+            InternalDatasetDetails dsDetails = (InternalDatasetDetails) ds.getDatasetDetails();
+            int sampleCardinalityTarget = stmtAnalyze.getSampleSize();
+            long sampleSeed = stmtAnalyze.getOrCreateSampleSeed();
+
+            Index.SampleIndexDetails newIndexDetailsPendingAdd =
+                    new Index.SampleIndexDetails(dsDetails.getPrimaryKey(), dsDetails.getKeySourceIndicator(),
+                            dsDetails.getPrimaryKeyType(), sampleCardinalityTarget, 0, 0, sampleSeed);
+            newIndexPendingAdd = new Index(dataverseName, datasetName, newIndexName, sampleIndexType,
+                    newIndexDetailsPendingAdd, false, false, MetadataUtil.PENDING_ADD_OP);
+
+            // #. add a new index with PendingAddOp
+            MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), newIndexPendingAdd);
+            // #. prepare to create the index artifact in NC.
+            JobSpecification spec =
+                    IndexUtil.buildSecondaryIndexCreationJobSpec(ds, newIndexPendingAdd, metadataProvider, sourceLoc);
+            if (spec == null) {
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Failed to create job spec for creating index '" + ds.getDatasetName() + "."
+                                + newIndexPendingAdd.getIndexName() + "'");
+            }
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
+            progressNewIndexCreate = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
+
+            // #. create the index artifact in NC.
+            runJob(hcc, spec);
+
+            // #. flush dataset
+            FlushDatasetUtil.flushDataset(hcc, metadataProvider, dataverseName, datasetName);
+
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            bActiveTxn = true;
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
+
+            // #. load data into the index in NC.
+            spec = IndexUtil.buildSecondaryIndexLoadingJobSpec(ds, newIndexPendingAdd, metadataProvider, sourceLoc);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
+
+            List<IOperatorStats> opStats = runJob(hcc, spec, jobFlags,
+                    Collections.singletonList(SampleOperationsHelper.DATASET_STATS_OPERATOR_NAME));
+            if (opStats == null || opStats.size() == 0) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, "", sourceLoc);
+            }
+            StreamStats stats = new StreamStats(opStats.get(0));
+
+            Index.SampleIndexDetails newIndexDetailsFinal = new Index.SampleIndexDetails(dsDetails.getPrimaryKey(),
+                    dsDetails.getKeySourceIndicator(), dsDetails.getPrimaryKeyType(), sampleCardinalityTarget,
+                    stats.getCardinality(), stats.getAvgTupleSize(), sampleSeed);
+            Index newIndexFinal = new Index(dataverseName, datasetName, newIndexName, sampleIndexType,
+                    newIndexDetailsFinal, false, false, MetadataUtil.PENDING_NO_OP);
+
+            // #. begin new metadataTxn
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            bActiveTxn = true;
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
+            // #. add same new index with PendingNoOp after deleting its entry with PendingAddOp
+            MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(),
+                    newIndexPendingAdd.getDataverseName(), newIndexPendingAdd.getDatasetName(),
+                    newIndexPendingAdd.getIndexName());
+            MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), newIndexFinal);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
+            progressNewIndexCreate = ProgressState.NO_PROGRESS;
+
+            if (existingIndex != null) {
+                // #. set existing index to PendingDropOp because we'll be dropping it next
+                mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+                bActiveTxn = true;
+                metadataProvider.setMetadataTxnContext(mdTxnCtx);
+                MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(),
+                        existingIndex.getDataverseName(), existingIndex.getDatasetName(), existingIndex.getIndexName());
+                existingIndex.setPendingOp(MetadataUtil.PENDING_DROP_OP);
+                MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), existingIndex);
+                existingIndexDropSpec = IndexUtil.buildDropIndexJobSpec(existingIndex, metadataProvider, ds, sourceLoc);
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                progressExistingIndexDrop = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
+                bActiveTxn = false;
+
+                // #. drop existing index on NCs
+                runJob(hcc, existingIndexDropSpec);
+
+                // #. drop existing index metadata
+                mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+                bActiveTxn = true;
+                metadataProvider.setMetadataTxnContext(mdTxnCtx);
+                MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(),
+                        existingIndex.getDataverseName(), existingIndex.getDatasetName(), existingIndex.getIndexName());
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                bActiveTxn = false;
+                progressExistingIndexDrop = ProgressState.NO_PROGRESS;
+            }
+
+        } catch (Exception e) {
+            LOGGER.error("failed to analyze dataset; executing compensating operations", e);
+            if (bActiveTxn) {
+                abort(e, e, mdTxnCtx);
+            }
+
+            if (progressExistingIndexDrop == ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA) {
+                // #. execute compensation operations remove the index in NCs
+                try {
+                    runJob(hcc, existingIndexDropSpec);
+                } catch (Exception e2) {
+                    // do no throw exception since still the metadata needs to be compensated.
+                    e.addSuppressed(e2);
+                }
+                // #. remove the record from the metadata.
+                mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+                metadataProvider.setMetadataTxnContext(mdTxnCtx);
+                try {
+                    MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(),
+                            existingIndex.getDataverseName(), existingIndex.getDatasetName(),
+                            existingIndex.getIndexName());
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                } catch (Exception e2) {
+                    e.addSuppressed(e2);
+                    abort(e, e2, mdTxnCtx);
+                    throw new IllegalStateException("System is inconsistent state: pending index("
+                            + existingIndex.getDataverseName() + "." + existingIndex.getDatasetName() + "."
+                            + existingIndex.getIndexName() + ") couldn't be removed from the metadata", e);
+                }
+            } else if (progressNewIndexCreate == ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA) {
+                // #. execute compensation operations remove the index in NCs
+                mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+                bActiveTxn = true;
+                metadataProvider.setMetadataTxnContext(mdTxnCtx);
+                try {
+                    JobSpecification jobSpec =
+                            IndexUtil.buildDropIndexJobSpec(newIndexPendingAdd, metadataProvider, ds, sourceLoc);
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                    bActiveTxn = false;
+                    runJob(hcc, jobSpec);
+                } catch (Exception e2) {
+                    e.addSuppressed(e2);
+                    if (bActiveTxn) {
+                        abort(e, e2, mdTxnCtx);
+                    }
+                }
+                // #. remove the record from the metadata.
+                mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+                metadataProvider.setMetadataTxnContext(mdTxnCtx);
+                try {
+                    MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(),
+                            newIndexPendingAdd.getDataverseName(), newIndexPendingAdd.getDatasetName(),
+                            newIndexPendingAdd.getIndexName());
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                } catch (Exception e2) {
+                    e.addSuppressed(e2);
+                    abort(e, e2, mdTxnCtx);
+                    throw new IllegalStateException("System is in inconsistent state: pending index("
+                            + newIndexPendingAdd.getDataverseName() + "." + newIndexPendingAdd.getDatasetName() + "."
+                            + newIndexPendingAdd.getIndexName() + ") couldn't be removed from the metadata", e);
+                }
+            }
+
+            throw e;
+        }
+    }
+
+    protected void handleAnalyzeDropStatement(MetadataProvider metadataProvider, Statement stmt,
+            IHyracksClientConnection hcc, IRequestParameters requestParams) throws Exception {
+        AnalyzeDropStatement analyzeDropStmt = (AnalyzeDropStatement) stmt;
+        metadataProvider.validateDatabaseObjectName(analyzeDropStmt.getDataverseName(),
+                analyzeDropStmt.getDatasetName(), analyzeDropStmt.getSourceLocation());
+        DataverseName dataverseName = getActiveDataverseName(analyzeDropStmt.getDataverseName());
+        String datasetName = analyzeDropStmt.getDatasetName();
+        if (isCompileOnly()) {
+            return;
+        }
+        lockUtil.analyzeDatasetDropBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
+        try {
+            doAnalyzeDatasetDrop(metadataProvider, analyzeDropStmt, dataverseName, datasetName, hcc, requestParams);
+        } finally {
+            metadataProvider.getLocks().unlock();
+        }
+    }
+
+    protected boolean doAnalyzeDatasetDrop(MetadataProvider metadataProvider, AnalyzeDropStatement stmtIndexDrop,
+            DataverseName dataverseName, String datasetName, IHyracksClientConnection hcc,
+            IRequestParameters requestParams) throws Exception {
+        SourceLocation sourceLoc = stmtIndexDrop.getSourceLocation();
+        Pair<String, String> sampleIndexNames = IndexUtil.getSampleIndexNames(datasetName);
+        String indexName1 = sampleIndexNames.first;
+        String indexName2 = sampleIndexNames.second;
+        ProgressState progress = ProgressState.NO_PROGRESS;
+        List<JobSpecification> jobsToExecute = new ArrayList<>();
+
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        boolean bActiveTxn = true;
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        boolean index1Exists = false, index2Exists = false;
+        try {
+            Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
+            if (ds == null) {
+                throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                        dataverseName);
+            }
+            if (ds.getDatasetType() != DatasetType.INTERNAL) {
+                throw new CompilationException(ErrorCode.OPERATION_NOT_SUPPORTED, sourceLoc);
+            }
+            Index index1 = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, indexName1);
+            Index index2 = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, indexName2);
+            index1Exists = index1 != null;
+            index2Exists = index2 != null;
+            if (!index1Exists && !index2Exists) {
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                return false;
+            }
+            ensureNonPrimaryIndexesDrop(index1, index2, sourceLoc);
+            validateDatasetState(metadataProvider, ds, sourceLoc);
+            prepareIndexDrop(metadataProvider, dataverseName, datasetName, sourceLoc, indexName1, jobsToExecute,
+                    mdTxnCtx, ds, index1);
+            prepareIndexDrop(metadataProvider, dataverseName, datasetName, sourceLoc, indexName2, jobsToExecute,
+                    mdTxnCtx, ds, index2);
+
+            // #. commit the existing transaction before calling runJob.
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
+            progress = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
+
+            for (JobSpecification jobSpec : jobsToExecute) {
+                runJob(hcc, jobSpec);
+            }
+
+            // #. begin a new transaction
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            bActiveTxn = true;
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
+
+            // #. finally, delete the existing indexes
+            if (index1Exists) {
+                MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName1);
+            }
+            if (index2Exists) {
+                MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName2);
+            }
+
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            return true;
+        } catch (Exception e) {
+            if (bActiveTxn) {
+                abort(e, e, mdTxnCtx);
+            }
+
+            if (progress == ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA) {
+                // #. execute compensation operations remove the all indexes in NC
+                try {
+                    for (JobSpecification jobSpec : jobsToExecute) {
+                        runJob(hcc, jobSpec);
+                    }
+                } catch (Exception e2) {
+                    // do no throw exception since still the metadata needs to be compensated.
+                    e.addSuppressed(e2);
+                }
+
+                // remove the record from the metadata.
+                mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+                metadataProvider.setMetadataTxnContext(mdTxnCtx);
+                try {
+                    if (index1Exists) {
+                        MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+                                datasetName, indexName1);
+                    }
+                    if (index2Exists) {
+                        MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+                                datasetName, indexName2);
+                    }
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                } catch (Exception e2) {
+                    e.addSuppressed(e2);
+                    abort(e, e2, mdTxnCtx);
+                    String msg = String.format(
+                            "System is in inconsistent state: pending index %1$s.%2$s.%3$s and/or %1$s.%2$s.%4$s "
+                                    + "couldn't be removed from the metadata",
+                            dataverseName, datasetName, indexName1, indexName2);
+                    throw new IllegalStateException(msg, e);
+                }
+            }
+
+            throw e;
+        }
+    }
+
+    private void ensureNonPrimaryIndexesDrop(Index index1, Index index2, SourceLocation sourceLoc)
+            throws AlgebricksException {
+        if (index1 != null) {
+            ensureNonPrimaryIndexDrop(index1, sourceLoc);
+        }
+        if (index2 != null) {
+            ensureNonPrimaryIndexDrop(index2, sourceLoc);
+        }
+    }
+
+    private void prepareIndexDrop(MetadataProvider metadataProvider, DataverseName dataverseName, String datasetName,
+            SourceLocation sourceLoc, String indexName, List<JobSpecification> jobsToExecute,
+            MetadataTransactionContext mdTxnCtx, Dataset ds, Index index) throws AlgebricksException {
+        if (index != null) {
+            // #. prepare a job to drop the index in NC.
+            jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, ds, sourceLoc));
+
+            // #. mark PendingDropOp on the existing index
+            MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
+            MetadataManager.INSTANCE.addIndex(mdTxnCtx,
+                    new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getIndexDetails(),
+                            index.isEnforced(), index.isPrimaryIndex(), MetadataUtil.PENDING_DROP_OP));
+        }
+    }
+
     protected void handleCompactStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
         CompactStatement compactStatement = (CompactStatement) stmt;
         SourceLocation sourceLoc = compactStatement.getSourceLocation();
         DataverseName dataverseName = getActiveDataverseName(compactStatement.getDataverseName());
         String datasetName = compactStatement.getDatasetName().getValue();
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -4208,6 +4694,7 @@
             }
         };
         final IStatementCompiler compiler = () -> {
+            long compileStart = System.nanoTime();
             MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
             boolean bActiveTxn = true;
             metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -4218,8 +4705,9 @@
                 stats.updateTotalWarningsCount(warningCollector.getTotalWarningsCount());
                 afterCompile();
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                stats.setCompileTime(System.nanoTime() - compileStart);
                 bActiveTxn = false;
-                return query.isExplain() || !sessionConfig.isExecuteQuery() ? null : jobSpec;
+                return query.isExplain() || isCompileOnly() ? null : jobSpec;
             } catch (Exception e) {
                 LOGGER.log(Level.INFO, e.getMessage(), e);
                 if (bActiveTxn) {
@@ -4277,8 +4765,8 @@
     private void updateJobStats(JobId jobId, Stats stats, ResultSetId rsId) throws HyracksDataException {
         final ClusterControllerService controllerService =
                 (ClusterControllerService) appCtx.getServiceContext().getControllerService();
-        org.apache.asterix.api.common.ResultMetadata resultMetadata =
-                (org.apache.asterix.api.common.ResultMetadata) controllerService.getResultDirectoryService()
+        org.apache.asterix.translator.ResultMetadata resultMetadata =
+                (org.apache.asterix.translator.ResultMetadata) controllerService.getResultDirectoryService()
                         .getResultMetadata(jobId, rsId);
         stats.setProcessedObjects(resultMetadata.getProcessedObjects());
         if (jobFlags.contains(JobFlag.PROFILE_RUNTIME)) {
@@ -4336,6 +4824,12 @@
         JobUtils.runJob(hcc, jobSpec, jobFlags, true);
     }
 
+    private static List<IOperatorStats> runJob(IHyracksClientConnection hcc, JobSpecification jobSpec,
+            EnumSet<JobFlag> jobFlags, List<String> statOperatorNames) throws Exception {
+        Pair<JobId, List<IOperatorStats>> p = JobUtils.runJob(hcc, jobSpec, jobFlags, true, statOperatorNames);
+        return p.second;
+    }
+
     private static void createAndRunJob(IHyracksClientConnection hcc, EnumSet<JobFlag> jobFlags, Mutable<JobId> jId,
             IStatementCompiler compiler, IMetadataLocker locker, ResultDelivery resultDelivery, IResultPrinter printer,
             IRequestParameters requestParameters, boolean cancellable, ICcApplicationContext appCtx,
@@ -4389,6 +4883,9 @@
         String ngName = stmtCreateNodegroup.getNodegroupName().getValue();
         metadataProvider.validateDatabaseObjectName(null, ngName, sourceLoc);
 
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockManager.acquireNodeGroupWriteLock(metadataProvider.getLocks(), ngName);
@@ -4423,9 +4920,6 @@
         DataverseName dataverseName = getActiveDataverseName(stmtRefresh.getDataverseName());
         String datasetName = stmtRefresh.getDatasetName().getValue();
         TransactionState transactionState = TransactionState.COMMIT;
-        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-        boolean bActiveTxn = true;
-        metadataProvider.setMetadataTxnContext(mdTxnCtx);
         JobSpecification spec = null;
         Dataset ds = null;
         List<ExternalFile> metadataFiles = null;
@@ -4436,6 +4930,12 @@
         Dataset transactionDataset = null;
         boolean lockAquired = false;
         boolean success = false;
+        if (isCompileOnly()) {
+            return;
+        }
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        boolean bActiveTxn = true;
         lockUtil.refreshDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
         try {
             ds = metadataProvider.findDataset(dataverseName, datasetName);
@@ -4770,7 +5270,7 @@
 
     private Map<VarIdentifier, IAObject> createExternalVariables(IReturningStatement stmt,
             Map<String, IAObject> stmtParams) throws CompilationException {
-        if (sessionConfig.isExecuteQuery()) {
+        if (!isCompileOnly()) {
             if (stmtParams == null || stmtParams.isEmpty()) {
                 return Collections.emptyMap();
             }
@@ -4795,6 +5295,10 @@
         }
     }
 
+    protected boolean isCompileOnly() {
+        return !sessionConfig.isExecuteQuery();
+    }
+
     protected void validateDatasetState(MetadataProvider metadataProvider, Dataset dataset, SourceLocation sourceLoc)
             throws Exception {
         validateIfResourceIsActiveInFeed(metadataProvider.getApplicationContext(), dataset, sourceLoc);
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 8f3deb1..2a66cfd 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
@@ -53,6 +53,7 @@
 import org.apache.asterix.api.http.server.VersionApiServlet;
 import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.app.cc.CCExtensionManager;
+import org.apache.asterix.app.cc.CcApplicationContext;
 import org.apache.asterix.app.config.ConfigValidator;
 import org.apache.asterix.app.io.PersistedResourceRegistry;
 import org.apache.asterix.app.replication.NcLifecycleCoordinator;
@@ -85,7 +86,6 @@
 import org.apache.asterix.metadata.lock.MetadataLockManager;
 import org.apache.asterix.metadata.utils.MetadataLockUtil;
 import org.apache.asterix.runtime.job.resource.JobCapacityController;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.asterix.translator.IStatementExecutorFactory;
 import org.apache.asterix.translator.Receptionist;
 import org.apache.asterix.util.MetadataBuiltinFunctions;
@@ -126,7 +126,7 @@
     protected WebManager webManager;
     protected ICcApplicationContext appCtx;
     private IJobCapacityController jobCapacityController;
-    private IHyracksClientConnection hcc;
+    private HyracksConnection hcc;
 
     @Override
     public void init(IServiceContext serviceCtx) throws Exception {
@@ -209,7 +209,7 @@
             IReceptionistFactory receptionistFactory, IConfigValidatorFactory configValidatorFactory,
             CCExtensionManager ccExtensionManager, IAdapterFactoryService adapterFactoryService)
             throws AlgebricksException, IOException {
-        return new CcApplicationContext(ccServiceCtx, getHcc(), () -> MetadataManager.INSTANCE, globalRecoveryManager,
+        return new CcApplicationContext(ccServiceCtx, hcc, () -> MetadataManager.INSTANCE, globalRecoveryManager,
                 lifecycleCoordinator, new ActiveNotificationHandler(), componentProvider, new MetadataLockManager(),
                 createMetadataLockUtil(), receptionistFactory, configValidatorFactory, ccExtensionManager,
                 adapterFactoryService);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
index 156b78a..48cf511 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
@@ -20,7 +20,7 @@
 
 import static java.util.regex.Pattern.CASE_INSENSITIVE;
 import static java.util.regex.Pattern.DOTALL;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.SECRET_ACCESS_KEY_FIELD_NAME;
 
 import java.util.regex.Pattern;
 
diff --git a/asterixdb/asterix-app/src/main/resources/cc.conf b/asterixdb/asterix-app/src/main/resources/cc.conf
index 98ede89..dc6e5a2 100644
--- a/asterixdb/asterix-app/src/main/resources/cc.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc.conf
@@ -52,6 +52,9 @@
 [common]
 log.dir = logs/
 log.level = INFO
+compiler.cbo=false
+compiler.cbotest=true
+compiler.queryplanshape=zigzag
 compiler.framesize=32KB
 compiler.sortmemory=320KB
 compiler.groupmemory=160KB
diff --git a/asterixdb/asterix-app/src/main/resources/cc2.conf b/asterixdb/asterix-app/src/main/resources/cc2.conf
index 016c485..c746198 100644
--- a/asterixdb/asterix-app/src/main/resources/cc2.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc2.conf
@@ -48,6 +48,9 @@
 [common]
 log.dir = logs/
 log.level = WARN
+compiler.cbo=false
+compiler.cbotest=true
+compiler.queryplanshape=zigzag
 compiler.framesize=32KB
 compiler.sortmemory=320KB
 compiler.groupmemory=160KB
diff --git a/asterixdb/asterix-app/src/main/resources/cc3.conf b/asterixdb/asterix-app/src/main/resources/cc3.conf
index 88362aa..8e9b5ab 100644
--- a/asterixdb/asterix-app/src/main/resources/cc3.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc3.conf
@@ -48,6 +48,9 @@
 [common]
 log.dir = logs/
 log.level = WARN
+compiler.cbo=false
+compiler.cbotest=true
+compiler.queryplanshape=zigzag
 compiler.framesize=32KB
 compiler.sortmemory=320KB
 compiler.groupmemory=160KB
diff --git a/asterixdb/asterix-app/src/main/resources/cc_no_cbo.conf b/asterixdb/asterix-app/src/main/resources/cc_no_cbo.conf
new file mode 100644
index 0000000..deb92e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/resources/cc_no_cbo.conf
@@ -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.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=asterix_nc1/iodevice1
+iodevices=asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=asterix_nc2/iodevice1
+iodevices=asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.pagesize=32KB
+storage.buffercache.size=128MB
+storage.memorycomponent.globalbudget=512MB
+storage.io.scheduler=greedy
+storage.filtered.memorycomponent.max.size=16MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+
+[common]
+log.dir = logs/
+log.level = INFO
+compiler.cbo=false
+compiler.cbotest=false
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+compiler.windowmemory=192KB
+compiler.sort.parallel=false
+compiler.internal.sanitycheck=true
+messaging.frame.size=4096
+messaging.frame.count=512
diff --git a/asterixdb/asterix-app/src/main/resources/entrypoint.py b/asterixdb/asterix-app/src/main/resources/entrypoint.py
index 7bad7ef..918596c 100755
--- a/asterixdb/asterix-app/src/main/resources/entrypoint.py
+++ b/asterixdb/asterix-app/src/main/resources/entrypoint.py
@@ -168,6 +168,7 @@
 
     def quit(self):
         self.alive = False
+        self.disconnect_sock()
         return True
 
     def handle_call(self):
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 86d8ed4..e8c2c1d 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
@@ -43,6 +43,7 @@
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.hyracks.bootstrap.CCApplication;
 import org.apache.asterix.hyracks.bootstrap.NCApplication;
+import org.apache.asterix.lang.common.util.ExpressionUtils;
 import org.apache.asterix.test.dataflow.TestLsmIoOpCallbackFactory;
 import org.apache.asterix.test.dataflow.TestPrimaryIndexOperationTrackerFactory;
 import org.apache.commons.io.FileUtils;
@@ -132,13 +133,13 @@
         cc = new ClusterControllerService(ccConfig, ccApplication);
 
         nodeNames = ccConfig.getConfigManager().getNodeNames();
-        if (deleteOldInstanceData) {
+        if (deleteOldInstanceData && nodeNames != null) {
             deleteTransactionLogs();
             removeTestStorageFiles();
             deleteCCFiles();
         }
         final List<NodeControllerService> nodeControllers = new ArrayList<>();
-        for (String nodeId : nodeNames) {
+        for (String nodeId : ExpressionUtils.emptyIfNull(nodeNames)) {
             // mark this NC as virtual, so that the CC doesn't try to start via NCService...
             configManager.set(nodeId, NCConfig.Option.NCSERVICE_PORT, NCConfig.NCSERVICE_PORT_DISABLED);
             final INCApplication ncApplication = createNCApplication();
@@ -303,7 +304,7 @@
 
         stopCC(false);
 
-        if (deleteOldInstanceData) {
+        if (deleteOldInstanceData && nodeNames != null) {
             deleteTransactionLogs();
             removeTestStorageFiles();
             deleteCCFiles();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiServletTest.java
index 8dcfa26..eb63218 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiServletTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiServletTest.java
@@ -31,8 +31,8 @@
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.asterix.api.http.server.VersionApiServlet;
+import org.apache.asterix.app.cc.CcApplicationContext;
 import org.apache.asterix.common.config.BuildProperties;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
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 0b80881..34696b1 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
@@ -26,6 +26,7 @@
 import java.util.List;
 import java.util.Map;
 
+import org.apache.asterix.app.cc.CcApplicationContext;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.app.nc.TransactionSubsystem;
 import org.apache.asterix.common.config.TransactionProperties;
@@ -61,7 +62,6 @@
 import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorNodePushable;
 import org.apache.asterix.runtime.operators.LSMPrimaryInsertOperatorNodePushable;
 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.runtime.CommitRuntime;
 import org.apache.asterix.transaction.management.service.logging.LogReader;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
index b80fa30..a2a3b48 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
@@ -26,6 +26,7 @@
 import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.asterix.app.cc.CcApplicationContext;
 import org.apache.asterix.app.replication.NcLifecycleCoordinator;
 import org.apache.asterix.app.replication.message.NCLifecycleTaskReportMessage;
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
@@ -37,7 +38,6 @@
 import org.apache.asterix.hyracks.bootstrap.CCApplication;
 import org.apache.asterix.runtime.transaction.ResourceIdManager;
 import org.apache.asterix.runtime.utils.BulkTxnIdFactory;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.ICCServiceContext;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
index c71e602..e5e33d0 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
@@ -34,6 +34,7 @@
 import org.apache.asterix.active.EntityId;
 import org.apache.asterix.active.NoRetryPolicyFactory;
 import org.apache.asterix.app.active.ActiveNotificationHandler;
+import org.apache.asterix.app.cc.CcApplicationContext;
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.cluster.IClusterStateManager;
@@ -53,7 +54,6 @@
 import org.apache.asterix.metadata.utils.MetadataLockUtil;
 import org.apache.asterix.runtime.functions.FunctionCollection;
 import org.apache.asterix.runtime.functions.FunctionManager;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.asterix.test.active.TestEventsListener.Behavior;
 import org.apache.asterix.test.base.TestMethodTracer;
 import org.apache.asterix.translator.IStatementExecutor;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
index c51fd95..cb123bf 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
@@ -37,6 +37,7 @@
 import org.apache.asterix.app.active.ActiveEntityEventsListener;
 import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.app.cc.CCExtensionManager;
+import org.apache.asterix.app.cc.CcApplicationContext;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.common.exceptions.ErrorCode;
@@ -45,7 +46,6 @@
 import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.asterix.test.runtime.ExecutionTestUtil;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.SessionOutput;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ProfilingTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ProfilingTestExecutor.java
new file mode 100644
index 0000000..44502da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ProfilingTestExecutor.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.test.common;
+
+import java.io.InputStream;
+import java.net.URI;
+import java.nio.charset.Charset;
+import java.util.List;
+import java.util.function.Predicate;
+
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.asterix.testframework.xml.ParameterTypeEnum;
+import org.apache.asterix.testframework.xml.TestCase;
+
+public class ProfilingTestExecutor extends TestExecutor {
+
+    private final TestCase.CompilationUnit.Parameter profile = new TestCase.CompilationUnit.Parameter();
+
+    public InputStream executeQueryService(String str, TestCaseContext.OutputFormat fmt, URI uri,
+            List<TestCase.CompilationUnit.Parameter> params, boolean jsonEncoded, Charset responseCharset,
+            Predicate<Integer> responseCodeValidator, boolean cancellable) throws Exception {
+        profile.setName("profile");
+        profile.setValue("timings");
+        profile.setType(ParameterTypeEnum.STRING);
+        params.add(profile);
+        return super.executeQueryService(str, fmt, uri, constructQueryParameters(str, fmt, params), jsonEncoded,
+                responseCharset, responseCodeValidator, cancellable);
+
+    }
+}
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 14784c5..1fd1af1 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
@@ -326,7 +326,7 @@
 
     public void runScriptAndCompareWithResult(File scriptFile, File expectedFile, File actualFile,
             ComparisonEnum compare, Charset actualEncoding, String statement) throws Exception {
-        LOGGER.info("Expected results file: {} ", canonicalize(expectedFile));
+        LOGGER.info("Expected results file: {} ", canonicalize(expectedFile.getAbsolutePath()));
         boolean regex = false;
         if (expectedFile.getName().endsWith(".ignore")) {
             return; //skip the comparison
@@ -404,9 +404,10 @@
             }
         } catch (Exception e) {
             if (!actualEncoding.equals(UTF_8)) {
-                LOGGER.info("Actual results file: {} encoding: {}", canonicalize(actualFile), actualEncoding);
+                LOGGER.info("Actual results file: {} encoding: {}", canonicalize(actualFile.getAbsolutePath()),
+                        actualEncoding);
             } else {
-                LOGGER.info("Actual results file: {}", canonicalize(actualFile));
+                LOGGER.info("Actual results file: {}", canonicalize(actualFile.getAbsolutePath()));
             }
             throw e;
         }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
index 2ee8fcf..fcc8bc6 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
@@ -86,7 +86,7 @@
                 filterCmpFactories, filterFields, opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory,
                 metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties,
                 true, bloomFilterFields, bloomFilterFalsePositiveRate, index.isPrimaryIndex(), btreeFields,
-                bloomFilterFields != null);
+                bloomFilterFields != null, false);
     }
 
     private static ITypeTraits[] getTypeTraits(MetadataProvider metadataProvider, Dataset dataset, Index index,
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
index 65cbb9e..d68586a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
@@ -27,8 +27,6 @@
 import java.io.StringReader;
 import java.nio.charset.StandardCharsets;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
 import java.util.Map;
 
 import org.apache.asterix.api.java.AsterixJavaClient;
@@ -63,7 +61,7 @@
 
     static {
         EXTENSION_RESULT = "plan.json";
-        PATH_ACTUAL = "target" + File.separator + "jplantest" + SEPARATOR;
+        PATH_ACTUAL = "target" + SEPARATOR + "jplantest" + SEPARATOR;
     }
 
     @Parameters(name = "JsonLogicalPlanTest {index}: {0}")
@@ -71,8 +69,8 @@
         return AbstractOptimizerTest.tests();
     }
 
-    public JsonLogicalPlanTest(final File queryFile, final File expectedFile, final File actualFile) {
-        super(queryFile, expectedFile, actualFile);
+    public JsonLogicalPlanTest(File queryFile, String expectedFilePath, File actualFile) {
+        super(queryFile, actualFile);
     }
 
     @Test
@@ -82,7 +80,7 @@
 
     @Override
     protected void runAndCompare(String query, ILangCompilationProvider provider, Map<String, IAObject> queryParams,
-            IHyracksClientConnection hcc, List<String> linesExpected) throws Exception {
+            IHyracksClientConnection hcc) throws Exception {
         FileUtils.writeStringToFile(actualFile, "", StandardCharsets.UTF_8);
         String planStr;
         try (PrintWriter plan = new PrintWriter(actualFile)) {
@@ -99,7 +97,7 @@
         }
 
         BufferedReader readerActual =
-                new BufferedReader(new InputStreamReader(new FileInputStream(actualFile), "UTF-8"));
+                new BufferedReader(new InputStreamReader(new FileInputStream(actualFile), StandardCharsets.UTF_8));
         String lineActual, objectActual = "";
         boolean firstPlan = false;
         while ((lineActual = readerActual.readLine()) != null) {
@@ -123,10 +121,4 @@
             readerActual.close();
         }
     }
-
-    @Override
-    protected List<String> getExpectedLines() {
-        // this test only checks the produced result is valid, so no expected results
-        return Collections.emptyList();
-    }
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonOptimizedLogicalPlanTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonOptimizedLogicalPlanTest.java
index 7f78743..9bf03ec 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonOptimizedLogicalPlanTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonOptimizedLogicalPlanTest.java
@@ -32,11 +32,11 @@
     static {
         EXTENSION_RESULT = "plan.json";
         optimized = true;
-        PATH_ACTUAL = "target" + File.separator + "joptplantest" + SEPARATOR;
+        PATH_ACTUAL = "target" + SEPARATOR + "joptplantest" + SEPARATOR;
     }
 
-    public JsonOptimizedLogicalPlanTest(File queryFile, File expectedFile, File actualFile) {
-        super(queryFile, expectedFile, actualFile);
+    public JsonOptimizedLogicalPlanTest(File queryFile, String expectedFilePath, File actualFile) {
+        super(queryFile, expectedFilePath, actualFile);
     }
 
     @Parameterized.Parameters(name = "JsonOptimizedLogicalPlanTest {index}: {0}")
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/AbstractOptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/AbstractOptimizerTest.java
index 0fba54b..8126d1e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/AbstractOptimizerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/AbstractOptimizerTest.java
@@ -19,12 +19,10 @@
 package org.apache.asterix.test.optimizer;
 
 import java.io.File;
-import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.List;
 import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -68,12 +66,11 @@
     protected static final String PATH_BASE =
             "src" + SEPARATOR + "test" + SEPARATOR + "resources" + SEPARATOR + "optimizerts" + SEPARATOR;
     protected static final String PATH_QUERIES = PATH_BASE + "queries" + SEPARATOR;
-    protected static final String PATH_EXPECTED = PATH_BASE + "results" + SEPARATOR;
     protected static String PATH_ACTUAL;
 
     protected static final ArrayList<String> ignore = AsterixTestHelper.readTestListFile(FILENAME_IGNORE, PATH_BASE);
     protected static final ArrayList<String> only = AsterixTestHelper.readTestListFile(FILENAME_ONLY, PATH_BASE);
-    protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
+    protected static String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
     protected static final ILangCompilationProvider sqlppCompilationProvider = new SqlppCompilationProvider();
     protected static ILangCompilationProvider extensionLangCompilationProvider = null;
     protected static IStatementExecutorFactory statementExecutorFactory = new DefaultStatementExecutorFactory();
@@ -123,9 +120,8 @@
         }
         if (file.isFile() && file.getName().endsWith(EXTENSION_SQLPP)) {
             String resultFileName = AsterixTestHelper.extToResExt(file.getName(), EXTENSION_RESULT);
-            File expectedFile = new File(PATH_EXPECTED + path + resultFileName);
             File actualFile = new File(PATH_ACTUAL + SEPARATOR + path + resultFileName);
-            testArgs.add(new Object[] { file, expectedFile, actualFile });
+            testArgs.add(new Object[] { file, path + resultFileName, actualFile });
         }
     }
 
@@ -143,20 +139,15 @@
     }
 
     protected final File actualFile;
-    protected final File expectedFile;
     protected final File queryFile;
 
-    public AbstractOptimizerTest(final File queryFile, final File expectedFile, final File actualFile) {
+    public AbstractOptimizerTest(final File queryFile, final File actualFile) {
         this.queryFile = queryFile;
-        this.expectedFile = expectedFile;
         this.actualFile = actualFile;
     }
 
     protected abstract void runAndCompare(String query, ILangCompilationProvider provider,
-            Map<String, IAObject> queryParams, IHyracksClientConnection hcc, List<String> linesExpected)
-            throws Exception;
-
-    protected abstract List<String> getExpectedLines() throws IOException;
+            Map<String, IAObject> queryParams, IHyracksClientConnection hcc) throws Exception;
 
     @Test
     public void test() throws Exception {
@@ -202,11 +193,10 @@
                 provider = extensionLangCompilationProvider;
             }
             IHyracksClientConnection hcc = integrationUtil.getHyracksClientConnection();
-            List<String> linesExpected = getExpectedLines();
             if (repeat) {
-                runAndRepeat(placeholder, substitutions, query, provider, queryParams, hcc, linesExpected);
+                runAndRepeat(placeholder, substitutions, query, provider, queryParams, hcc);
             } else {
-                runAndCompare(query, provider, queryParams, hcc, linesExpected);
+                runAndCompare(query, provider, queryParams, hcc);
             }
 
             LOGGER.info("Test \"" + queryFile.getPath() + "\" PASSED!");
@@ -222,16 +212,16 @@
     }
 
     private void runAndRepeat(String placeholder, JsonNode substitutions, String query,
-            ILangCompilationProvider provider, Map<String, IAObject> queryParams, IHyracksClientConnection hcc,
-            List<String> linesExpected) throws Exception {
+            ILangCompilationProvider provider, Map<String, IAObject> queryParams, IHyracksClientConnection hcc)
+            throws Exception {
         if (substitutions.isArray()) {
             for (int i = 0, size = substitutions.size(); i < size; i++) {
                 String substitute = substitutions.get(i).asText();
                 String newQuery = query.replaceAll(placeholder, substitute);
-                runAndCompare(newQuery, provider, queryParams, hcc, linesExpected);
+                runAndCompare(newQuery, provider, queryParams, hcc);
             }
         } else {
-            runAndCompare(query, provider, queryParams, hcc, linesExpected);
+            runAndCompare(query, provider, queryParams, hcc);
         }
     }
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/CBOOptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/CBOOptimizerTest.java
new file mode 100644
index 0000000..a5a99d9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/CBOOptimizerTest.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.asterix.test.optimizer;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collection;
+import java.util.List;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the optimization tests with CBO enabled. It compares a test's actual result against the expected result in
+ * {@link CBOOptimizerTest#CBO_PATH_EXPECTED} if one is provided. Otherwise, it compares against the expected result in
+ * {@link OptimizerTest#PATH_EXPECTED}.
+ */
+@RunWith(Parameterized.class)
+public class CBOOptimizerTest extends OptimizerTest {
+
+    private static final String CBO_PATH_EXPECTED = PATH_BASE + "results_cbo" + SEPARATOR;
+    static {
+        TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
+        EXTENSION_RESULT = "plan";
+        PATH_ACTUAL = "target" + SEPARATOR + "cbo_opttest" + SEPARATOR;
+    }
+
+    @Parameters(name = "CBOOptimizerTest {index}: {0}")
+    public static Collection<Object[]> tests() {
+        return AbstractOptimizerTest.tests();
+    }
+
+    public CBOOptimizerTest(File queryFile, String expectedFilePath, File actualFile) {
+        super(queryFile, expectedFilePath, actualFile);
+    }
+
+    @Override
+    protected List<String> getExpectedLines() throws IOException {
+        Path cboFilePath = Path.of(CBO_PATH_EXPECTED, expectedFilePath);
+        return Files.exists(cboFilePath) ? Files.readAllLines(cboFilePath, StandardCharsets.UTF_8)
+                : super.getExpectedLines();
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
index a89585d..e15774b 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
@@ -24,6 +24,7 @@
 import java.io.StringReader;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
+import java.nio.file.Path;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
@@ -51,12 +52,15 @@
 @RunWith(Parameterized.class)
 public class OptimizerTest extends AbstractOptimizerTest {
 
+    protected static final String PATH_EXPECTED = PATH_BASE + "results" + SEPARATOR;
     private static final String PATTERN_VAR_ID_PREFIX = "\\$\\$";
     private static final Pattern PATTERN_VAR_ID = Pattern.compile(PATTERN_VAR_ID_PREFIX + "(\\d+)");
+    protected String expectedFilePath;
 
     static {
+        TEST_CONFIG_FILE_NAME = "src/main/resources/cc_no_cbo.conf";
         EXTENSION_RESULT = "plan";
-        PATH_ACTUAL = "target" + File.separator + "opttest" + SEPARATOR;
+        PATH_ACTUAL = "target" + SEPARATOR + "opttest" + SEPARATOR;
     }
 
     @Parameters(name = "OptimizerTest {index}: {0}")
@@ -64,8 +68,9 @@
         return AbstractOptimizerTest.tests();
     }
 
-    public OptimizerTest(final File queryFile, final File expectedFile, final File actualFile) {
-        super(queryFile, expectedFile, actualFile);
+    public OptimizerTest(File queryFile, String expectedFilePath, File actualFile) {
+        super(queryFile, actualFile);
+        this.expectedFilePath = expectedFilePath;
     }
 
     @Test
@@ -75,7 +80,7 @@
 
     @Override
     protected void runAndCompare(String query, ILangCompilationProvider provider, Map<String, IAObject> queryParams,
-            IHyracksClientConnection hcc, List<String> linesExpected) throws Exception {
+            IHyracksClientConnection hcc) throws Exception {
         FileUtils.writeStringToFile(actualFile, "", StandardCharsets.UTF_8);
         try (PrintWriter plan = new PrintWriter(actualFile)) {
             AsterixJavaClient asterix = new AsterixJavaClient(
@@ -89,6 +94,7 @@
 
         List<String> linesActual = Files.readAllLines(actualFile.toPath(), StandardCharsets.UTF_8);
 
+        List<String> linesExpected = getExpectedLines();
         int varBaseExpected = findBaseVarId(linesExpected);
         int varBaseActual = findBaseVarId(linesActual);
 
@@ -116,9 +122,8 @@
         }
     }
 
-    @Override
     protected List<String> getExpectedLines() throws IOException {
-        return Files.readAllLines(expectedFile.toPath(), StandardCharsets.UTF_8);
+        return Files.readAllLines(Path.of(PATH_EXPECTED, expectedFilePath), StandardCharsets.UTF_8);
     }
 
     private boolean planLineEquals(String lineExpected, int varIdBaseExpected, String lineActual, int varIdBaseActual) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java
index 408882d..d704d8e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java
@@ -33,6 +33,7 @@
 import java.util.List;
 
 import org.apache.asterix.app.external.ExternalUDFLibrarian;
+import org.apache.asterix.app.external.IExternalUDFLibrarian;
 import org.apache.asterix.common.utils.StorageConstants;
 import org.apache.asterix.test.common.TestExecutor;
 import org.apache.asterix.testframework.context.TestCaseContext;
@@ -65,11 +66,17 @@
     }
 
     public static void setUp(String configFile, TestExecutor executor, boolean startHdfs) throws Exception {
+        setUp(configFile, executor, startHdfs, false, new ExternalUDFLibrarian());
+    }
+
+    public static void setUp(String configFile, TestExecutor executor, boolean startHdfs, boolean disableLangExec,
+            IExternalUDFLibrarian librarian) throws Exception {
         testExecutor = executor;
         File outdir = new File(PATH_ACTUAL);
         outdir.mkdirs();
-        ExecutionTestUtil.setUp(cleanupOnStart, configFile, integrationUtil, startHdfs, null);
-        librarian = new ExternalUDFLibrarian();
+        if (!disableLangExec) {
+            ExecutionTestUtil.setUp(cleanupOnStart, configFile, integrationUtil, startHdfs, null);
+        }
         testExecutor.setLibrarian(librarian);
         if (repeat != 1) {
             System.out.println("FYI: each test will be run " + repeat + " times.");
@@ -151,7 +158,9 @@
         NodeControllerService[] ncs = integrationUtil.ncs;
         // Checks that dataset files are uniformly distributed across each io device.
         for (NodeControllerService nc : ncs) {
-            checkNcStore(nc);
+            if (nc != null) {
+                checkNcStore(nc);
+            }
         }
     }
 
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/MetricsExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/MetricsExecutionTest.java
index a973c63..7be1640 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/MetricsExecutionTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/MetricsExecutionTest.java
@@ -31,7 +31,7 @@
 import org.junit.runners.Parameterized.Parameters;
 
 /**
- * Runs the cluster state runtime tests with the storage parallelism.
+ * Runs the cluster runtime tests and checks the query metrics.
  */
 @RunWith(Parameterized.class)
 public class MetricsExecutionTest {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppBatchPointLookupExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppBatchPointLookupExecutionTest.java
new file mode 100644
index 0000000..2608447
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppBatchPointLookupExecutionTest.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.hyracks.control.nc.NodeControllerService;
+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 SQL++ with Batched point-lookups enabled.
+ */
+@RunWith(Parameterized.class)
+public class SqlppBatchPointLookupExecutionTest {
+    protected static final String TEST_CONFIG_FILE_NAME = "src/test/resources/cc-batch-lookup.conf";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        final TestExecutor testExecutor = new TestExecutor();
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor);
+        setNcEndpoints(testExecutor);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "SqlppBatchPointLookupExecutionTest {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only_batch_lookup.xml", "testsuite_sqlpp_batch_lookup.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public SqlppBatchPointLookupExecutionTest(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+
+    private static void setNcEndpoints(TestExecutor testExecutor) {
+        final NodeControllerService[] ncs = ExecutionTestUtil.integrationUtil.ncs;
+        final Map<String, InetSocketAddress> ncEndPoints = new HashMap<>();
+        final String ip = InetAddress.getLoopbackAddress().getHostAddress();
+        for (NodeControllerService nc : ncs) {
+            final String nodeId = nc.getId();
+            final INcApplicationContext appCtx = (INcApplicationContext) nc.getApplicationContext();
+            int apiPort = appCtx.getExternalProperties().getNcApiPort();
+            ncEndPoints.put(nodeId, InetSocketAddress.createUnresolved(ip, apiPort));
+        }
+        testExecutor.setNcEndPoints(ncEndPoints);
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppProfiledExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppProfiledExecutionTest.java
new file mode 100644
index 0000000..b056d2c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppProfiledExecutionTest.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.test.runtime;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.test.common.ProfilingTestExecutor;
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.hyracks.control.nc.NodeControllerService;
+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 SQL++ runtime tests with full runtime profiling.
+ */
+@RunWith(Parameterized.class)
+public class SqlppProfiledExecutionTest {
+    protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        final TestExecutor testExecutor = new ProfilingTestExecutor();
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor);
+        setNcEndpoints(testExecutor);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "SqlppProfiledExecutionTest {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only_sqlpp.xml", "testsuite_sqlpp_profiled.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public SqlppProfiledExecutionTest(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+
+    private static void setNcEndpoints(TestExecutor testExecutor) {
+        final NodeControllerService[] ncs = ExecutionTestUtil.integrationUtil.ncs;
+        final Map<String, InetSocketAddress> ncEndPoints = new HashMap<>();
+        final String ip = InetAddress.getLoopbackAddress().getHostAddress();
+        for (NodeControllerService nc : ncs) {
+            final String nodeId = nc.getId();
+            final INcApplicationContext appCtx = (INcApplicationContext) nc.getApplicationContext();
+            int apiPort = appCtx.getExternalProperties().getNcApiPort();
+            ncEndPoints.put(nodeId, InetSocketAddress.createUnresolved(ip, apiPort));
+        }
+        testExecutor.setNcEndPoints(ncEndPoints);
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
index 8a87de7..5d10028 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
@@ -279,6 +279,7 @@
         invokeMethod(rewriter, "rewriteGroupBys");
         invokeMethod(rewriter, "rewriteSetOperations");
         invokeMethod(rewriter, "inlineColumnAlias");
+        invokeMethod(rewriter, "rewriteSelectExcludeSugar");
         invokeMethod(rewriter, "rewriteWindowExpressions");
         invokeMethod(rewriter, "rewriteGroupingSets");
         invokeMethod(rewriter, "variableCheckAndRewrite");
diff --git a/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf b/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf
new file mode 100644
index 0000000..6e10481
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf
@@ -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.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=target/tmp/asterix_nc1/iodevice1,
+iodevices=../asterix-server/target/tmp/asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+credential.file=src/test/resources/security/passwd
+python.cmd.autolocate=true
+python.env=FOO=BAR=BAZ,BAR=BAZ
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.pagesize=32KB
+storage.buffercache.size=128MB
+storage.memorycomponent.globalbudget=512MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+credential.file=src/test/resources/security/passwd
+
+[common]
+log.dir = logs/
+log.level = INFO
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+compiler.windowmemory=192KB
+compiler.batch.lookup=true
+messaging.frame.size=4096
+messaging.frame.count=512
+
diff --git a/asterixdb/asterix-app/src/test/resources/cc-compression.conf b/asterixdb/asterix-app/src/test/resources/cc-compression.conf
index c8d9780..373d9bb 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-compression.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-compression.conf
@@ -47,6 +47,8 @@
 
 [common]
 log.level = INFO
+compiler.cbo=false
+compiler.cbotest=true
 compiler.framesize=32KB
 compiler.sortmemory=320KB
 compiler.groupmemory=160KB
diff --git a/asterixdb/asterix-app/src/test/resources/cc-ssl.conf b/asterixdb/asterix-app/src/test/resources/cc-ssl.conf
index 499e9fc..ff7e73b 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-ssl.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-ssl.conf
@@ -59,6 +59,8 @@
 [common]
 log.dir = logs/
 log.level = INFO
+compiler.cbo=false
+compiler.cbotest=true
 compiler.framesize=32KB
 compiler.sortmemory=320KB
 compiler.groupmemory=160KB
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query4.sqlpp
new file mode 100644
index 0000000..fae9ea4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query4.sqlpp
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: ASTERIXDB-3043. Part of q18 of tpcds.
+ */
+
+DROP DATAVERSE tpcds IF EXISTS;
+CREATE DATAVERSE tpcds;
+
+USE tpcds;
+
+CREATE TYPE tpcds.catalog_sales_type AS
+ CLOSED {
+  cs_sold_date_sk:           bigint?,
+  cs_sold_time_sk:           bigint?,
+  cs_ship_date_sk:           bigint?,
+  cs_bill_customer_sk:       bigint?,
+  cs_bill_cdemo_sk:          bigint?,
+  cs_bill_hdemo_sk:          bigint?,
+  cs_bill_addr_sk:           bigint?,
+  cs_ship_customer_sk:       bigint?,
+  cs_ship_cdemo_sk:          bigint?,
+  cs_ship_hdemo_sk:          bigint?,
+  cs_ship_addr_sk:           bigint?,
+  cs_call_center_sk:         bigint?,
+  cs_catalog_page_sk:        bigint?,
+  cs_ship_mode_sk:           bigint?,
+  cs_warehouse_sk:           bigint?,
+  cs_item_sk:                bigint,
+  cs_promo_sk:               bigint?,
+  cs_order_number:           bigint,
+  cs_quantity:               bigint?,
+  cs_wholesale_cost:         double?,
+  cs_list_price:             double?,
+  cs_sales_price:            double?,
+  cs_ext_discount_amt:       double?,
+  cs_ext_sales_price:        double?,
+  cs_ext_wholesale_cost:     double?,
+  cs_ext_list_price:         double?,
+  cs_ext_tax:                double?,
+  cs_coupon_amt:             double?,
+  cs_ext_ship_cost:          double?,
+  cs_net_paid:               double?,
+  cs_net_paid_inc_tax:       double?,
+  cs_net_paid_inc_ship:      double?,
+  cs_net_paid_inc_ship_tax:  double?,
+  cs_net_profit:             double?
+};
+
+CREATE TYPE tpcds.customer_demographics_type AS
+ CLOSED {
+  cd_demo_sk : int64,
+  cd_gender : string?,
+  cd_marital_status : string?,
+  cd_education_status : string?,
+  cd_purchase_estimate : int64?,
+  cd_credit_rating : string?,
+  cd_dep_count : int64?,
+  cd_dep_employed_count : int64?,
+  cd_dep_college_count : int64?
+};
+
+CREATE TYPE tpcds.customer_type AS
+ CLOSED {
+  c_customer_sk : int64,
+  c_customer_id : string,
+  c_current_cdemo_sk : int64?,
+  c_current_hdemo_sk : int64?,
+  c_current_addr_sk : int64?,
+  c_first_shipto_date_sk : int64?,
+  c_first_sales_date_sk : int64?,
+  c_salutation : string?,
+  c_first_name : string?,
+  c_last_name : string?,
+  c_preferred_cust_flag : string?,
+  c_birth_day : int64?,
+  c_birth_month : int64?,
+  c_birth_year : int64?,
+  c_birth_country : string?,
+  c_login : string?,
+  c_email_address : string?,
+  c_last_review_date : string?
+};
+
+CREATE TYPE tpcds.customer_address_type AS
+ CLOSED {
+  ca_address_sk : bigint,
+  ca_address_id : string,
+  ca_street_number : string?,
+  ca_street_name : string?,
+  ca_street_type : string?,
+  ca_suite_number : string?,
+  ca_city : string?,
+  ca_county : string?,
+  ca_state : string?,
+  ca_zip : string?,
+  ca_country : string?,
+  ca_gmt_offset : double?,
+  ca_location_type : string?
+ };
+
+CREATE TYPE tpcds.item_type AS
+ CLOSED {
+  i_item_sk : bigint,
+  i_item_id : string,
+  i_rec_start_date : string?,
+  i_rec_end_date : string?,
+  i_item_desc : string?,
+  i_current_price : double?,
+  i_wholesale_cost : double?,
+  i_brand_id : bigint?,
+  i_brand : string?,
+  i_class_id : bigint?,
+  i_class : string?,
+  i_category_id : bigint?,
+  i_category : string?,
+  i_manufact_id : bigint?,
+  i_manufact : string?,
+  i_size : string?,
+  i_formulation : string?,
+  i_color : string?,
+  i_units : string?,
+  i_container : string?,
+  i_manager_id : bigint?,
+  i_product_name : string?
+};
+
+CREATE TYPE tpcds.date_dim_type AS
+ CLOSED {
+  d_date_sk : bigint,
+  d_date_id : string,
+  d_date : string?,
+  d_month_seq : bigint?,
+  d_week_seq : bigint?,
+  d_quarter_seq : bigint?,
+  d_year : bigint? ,
+  d_dow : bigint? ,
+  d_moy : bigint?,
+  d_dom : bigint?,
+  d_qoy : bigint?,
+  d_fy_year : bigint?,
+  d_fy_quarter_seq : bigint?,
+  d_fy_week_seq : bigint?,
+  d_day_name : string?,
+  d_quarter_name : string?,
+  d_holiday : string?,
+  d_weekend : string?,
+  d_following_holiday : string?,
+  d_first_dom : bigint?,
+  d_last_dom : bigint?,
+  d_same_day_ly : bigint?,
+  d_same_day_lq : bigint?,
+  d_current_day : string?,
+  d_current_week : string?,
+  d_current_month : string?,
+  d_current_quarter : string?,
+  d_current_year : string?
+};
+
+CREATE DATASET catalog_sales(catalog_sales_type) PRIMARY KEY cs_item_sk, cs_order_number;
+
+CREATE DATASET customer_demographics(customer_demographics_type) PRIMARY KEY cd_demo_sk;
+
+CREATE DATASET customer(customer_type) PRIMARY KEY c_customer_sk;
+
+CREATE DATASET customer_address(customer_address_type) PRIMARY KEY ca_address_sk;
+
+CREATE DATASET item(item_type) PRIMARY KEY i_item_sk;
+
+CREATE DATASET date_dim(date_dim_type) PRIMARY KEY d_date_sk;
+
+SELECT count (*)
+FROM customer c, customer_demographics cd2, customer_address ca
+WHERE
+ c.c_current_cdemo_sk  /*+ indexnl */ = cd2.cd_demo_sk
+ AND c.c_current_addr_sk  /*+ indexnl */= ca.ca_address_sk
+ AND c.c_birth_month in [4,5];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/extract-common-operators/extract-common-operators.01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/extract-common-operators/extract-common-operators.01.sqlpp
new file mode 100644
index 0000000..11c7325
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/extract-common-operators/extract-common-operators.01.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.
+ */
+
+/*
+ * test fix for ASTERIXDB-3075
+ */
+
+LET dv_name = ["test2", "test1"], ds_name = ["ds2", "ds1"], syn_name = [ "syn2", "syn1" ],
+synonym_names = (SELECT s.SynonymName, s.ObjectName
+                 FROM Metadata.`Synonym` s, syn_name
+                 WHERE s.SynonymName = syn_name),
+
+dataset_ds_dv_names = (SELECT d.DatasetName, d.DataverseName
+                       FROM Metadata.`Dataset` d, (SELECT * FROM ds_name, dv_name) AS ds_dv_names
+                       WHERE d.DatasetName = ds_dv_names.ds_name AND d.DataverseName = ds_dv_names.dv_name),
+
+dataset_dv_ds_names = (SELECT d.DatasetName, d.DataverseName
+                       FROM Metadata.`Dataset` d, (SELECT * FROM dv_name, ds_name) AS dv_ds_names
+                       WHERE d.DatasetName = dv_ds_names.ds_name AND d.DataverseName = dv_ds_names.dv_name),
+
+left_branch = (SELECT s.SynonymName, s.DataverseName, s.ObjectName
+               FROM Metadata.`Synonym` s LEFT OUTER JOIN dataset_dv_ds_names
+               ON dataset_dv_ds_names.DatasetName = s.ObjectName
+               ORDER BY s.DataverseName, s.SynonymName),
+
+right_branch = (SELECT synonym_names.SynonymName
+                FROM synonym_names LEFT OUTER JOIN dataset_ds_dv_names
+                ON dataset_ds_dv_names.DatasetName = synonym_names.ObjectName)
+
+FROM left_branch lb LEFT OUTER JOIN right_branch rb
+ON lb.ObjectName = rb.SynonymName
+SELECT lb.DataverseName, lb.SynonymName
+ORDER BY lb.DataverseName, lb.SynonymName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/nlj_partitioning_property_1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/nlj_partitioning_property_1.sqlpp
new file mode 100644
index 0000000..e0b4ba4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/nlj_partitioning_property_1.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.
+ */
+
+/*
+ * Test the fix for nested loop join delivered partitioning property ASTERIXDB-3066
+ */
+
+DROP DATAVERSE tpch IF EXISTS;
+CREATE DATAVERSE tpch;
+
+USE tpch;
+
+CREATE TYPE tpch.SupplierType AS
+ CLOSED {
+  s_suppkey : bigint,
+  s_name : string,
+  s_address : string,
+  s_nationkey : bigint,
+  s_phone : string,
+  s_acctbal : double,
+  s_comment : string
+};
+
+CREATE TYPE tpch.PartType AS
+ CLOSED {
+  p_partkey : bigint,
+  p_name : string,
+  p_mfgr : string,
+  p_brand : string,
+  p_type : string,
+  p_size : bigint,
+  p_container : string,
+  p_retailprice : double,
+  p_comment : string
+};
+
+CREATE TYPE tpch.PartSuppType AS
+ CLOSED {
+  ps_partkey : bigint,
+  ps_suppkey : bigint,
+  ps_availqty : bigint,
+  ps_supplycost : double,
+  ps_comment : string
+};
+
+CREATE DATASET Supplier(SupplierType) PRIMARY KEY s_suppkey;
+CREATE DATASET Part(PartType) PRIMARY KEY p_partkey;
+CREATE DATASET Partsupp(PartSuppType) PRIMARY KEY ps_partkey,ps_suppkey;
+
+CREATE INDEX partsupp_fk_part ON Partsupp (ps_partkey);
+CREATE INDEX partsupp_fk_supplier ON Partsupp (ps_suppkey);
+
+FROM Supplier s JOIN (FROM Partsupp ps, Part p SELECT ps.ps_suppkey AS suppkey) AS p_ps
+ON s.s_suppkey /*+ hash-bcast */ = p_ps.suppkey
+SELECT count(*) AS count;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/nlj_partitioning_property_2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/nlj_partitioning_property_2.sqlpp
new file mode 100644
index 0000000..8b9a5bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/nlj_partitioning_property_2.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.
+ */
+
+/*
+ * Test the fix for nested loop join delivered partitioning property ASTERIXDB-3066
+ */
+
+DROP DATAVERSE tpch IF EXISTS;
+CREATE DATAVERSE tpch;
+
+USE tpch;
+
+CREATE TYPE tpch.SupplierType AS
+ CLOSED {
+  s_suppkey : bigint,
+  s_name : string,
+  s_address : string,
+  s_nationkey : bigint,
+  s_phone : string,
+  s_acctbal : double,
+  s_comment : string
+};
+
+CREATE TYPE tpch.PartType AS
+ CLOSED {
+  p_partkey : bigint,
+  p_name : string,
+  p_mfgr : string,
+  p_brand : string,
+  p_type : string,
+  p_size : bigint,
+  p_container : string,
+  p_retailprice : double,
+  p_comment : string
+};
+
+CREATE TYPE tpch.PartSuppType AS
+ CLOSED {
+  ps_partkey : bigint,
+  ps_suppkey : bigint,
+  ps_availqty : bigint,
+  ps_supplycost : double,
+  ps_comment : string
+};
+
+CREATE DATASET Supplier(SupplierType) PRIMARY KEY s_suppkey;
+CREATE DATASET Part(PartType) PRIMARY KEY p_partkey;
+CREATE DATASET Partsupp(PartSuppType) PRIMARY KEY ps_partkey,ps_suppkey;
+
+CREATE INDEX partsupp_fk_part ON Partsupp (ps_partkey);
+CREATE INDEX partsupp_fk_supplier ON Partsupp (ps_suppkey);
+
+USE tpch;
+
+FROM Supplier s JOIN (FROM Partsupp ps, Part p SELECT ps.ps_suppkey AS suppkey) AS p_ps ON TRUE
+SELECT count(*) AS count;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
index 0f996d1..7c8d8e7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
@@ -5,12 +5,12 @@
               {
                 -- AGGREGATE  |LOCAL|
                   -- ASSIGN  |LOCAL|
-                    -- MICRO_PRE_CLUSTERED_GROUP_BY[$$215]  |LOCAL|
+                    -- MICRO_PRE_CLUSTERED_GROUP_BY[$$230]  |LOCAL|
                             {
                               -- AGGREGATE  |LOCAL|
                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                             }
-                      -- MICRO_STABLE_SORT [$$215(ASC)]  |LOCAL|
+                      -- MICRO_STABLE_SORT [$$230(ASC)]  |LOCAL|
                         -- ASSIGN  |LOCAL|
                           -- UNNEST  |LOCAL|
                             -- SUBPLAN  |LOCAL|
@@ -27,7 +27,7 @@
             -- STREAM_PROJECT  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- INSERT_DELETE  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$199]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$214]  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
@@ -37,28 +37,28 @@
                                 -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- PRE_CLUSTERED_GROUP_BY[$$253]  |PARTITIONED|
+                                      -- PRE_CLUSTERED_GROUP_BY[$$268]  |PARTITIONED|
                                               {
                                                 -- AGGREGATE  |LOCAL|
                                                   -- STREAM_SELECT  |LOCAL|
                                                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                                               }
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STABLE_SORT [$$253(ASC)]  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$253]  |PARTITIONED|
+                                          -- STABLE_SORT [$$268(ASC)]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$268]  |PARTITIONED|
                                               -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- HYBRID_HASH_JOIN [$$309][$$222]  |PARTITIONED|
-                                                    -- HASH_PARTITION_EXCHANGE [$$309]  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$323][$$237]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$323]  |PARTITIONED|
                                                       -- RUNNING_AGGREGATE  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                           -- UNNEST  |PARTITIONED|
                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- PRE_CLUSTERED_GROUP_BY[$$317]  |PARTITIONED|
+                                                                -- PRE_CLUSTERED_GROUP_BY[$$326]  |PARTITIONED|
                                                                         {
                                                                           -- AGGREGATE  |LOCAL|
-                                                                            -- MICRO_PRE_CLUSTERED_GROUP_BY[$$319, $$321]  |LOCAL|
+                                                                            -- MICRO_PRE_CLUSTERED_GROUP_BY[$$328, $$330]  |LOCAL|
                                                                                     {
                                                                                       -- AGGREGATE  |LOCAL|
                                                                                         -- STREAM_SELECT  |LOCAL|
@@ -68,8 +68,8 @@
                                                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                                         }
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- STABLE_SORT [$$317(ASC), $$319(ASC), $$321(ASC)]  |PARTITIONED|
-                                                                      -- HASH_PARTITION_EXCHANGE [$$317]  |PARTITIONED|
+                                                                    -- STABLE_SORT [$$326(ASC), $$328(ASC), $$330(ASC)]  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$326]  |PARTITIONED|
                                                                         -- UNION_ALL  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- STREAM_PROJECT  |PARTITIONED|
@@ -86,7 +86,7 @@
                                                                                                   -- STREAM_PROJECT  |PARTITIONED|
                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                       -- RTREE_SEARCH (channels.Shelters.s_location)  |PARTITIONED|
-                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
                                                                                                           -- ASSIGN  |PARTITIONED|
                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                               -- NESTED_LOOP  |PARTITIONED|
@@ -107,7 +107,7 @@
                                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                               -- BTREE_SEARCH (channels.Reports.Reports)  |PARTITIONED|
                                                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                  -- STABLE_SORT [$$260(ASC)]  |PARTITIONED|
+                                                                                                                                  -- STABLE_SORT [$$275(ASC)]  |PARTITIONED|
                                                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                       -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -133,7 +133,7 @@
                                                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                 -- RTREE_SEARCH (channels.Shelters.s_location)  |PARTITIONED|
-                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
                                                                                                     -- ASSIGN  |PARTITIONED|
                                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                         -- NESTED_LOOP  |PARTITIONED|
@@ -154,7 +154,7 @@
                                                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                         -- BTREE_SEARCH (channels.Reports.Reports)  |PARTITIONED|
                                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                            -- STABLE_SORT [$$260(ASC)]  |PARTITIONED|
+                                                                                                                            -- STABLE_SORT [$$275(ASC)]  |PARTITIONED|
                                                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -168,12 +168,12 @@
                                                                                                                 -- DATASOURCE_SCAN (channels.UserLocations)  |PARTITIONED|
                                                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                    -- HASH_PARTITION_EXCHANGE [$$222]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$237]  |PARTITIONED|
                                                       -- ASSIGN  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- HYBRID_HASH_JOIN [$$233, $$235][$$224, $$225]  |PARTITIONED|
-                                                              -- HASH_PARTITION_EXCHANGE [$$233, $$235]  |PARTITIONED|
+                                                            -- HYBRID_HASH_JOIN [$$248, $$250][$$239, $$240]  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$248, $$250]  |PARTITIONED|
                                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                                   -- ASSIGN  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query4.plan
new file mode 100644
index 0000000..4e1a931
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query4.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$57][$$70]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (tpcds.customer_address.customer_address)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (tpcds.customer_demographics.customer_demographics)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$72]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (tpcds.customer)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan
index 132ec02..616e808 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan
@@ -34,7 +34,7 @@
                                                 }
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- STABLE_SORT [$$172(ASC)]  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$172]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- NESTED_LOOP  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/extract-common-operators/extract-common-operators.01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/extract-common-operators/extract-common-operators.01.plan
new file mode 100644
index 0000000..677a45f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/extract-common-operators/extract-common-operators.01.plan
@@ -0,0 +1,98 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$381(ASC), $$382(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$381(ASC), $$382(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$390][$$386]  |PARTITIONED|
+                    -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$381(ASC), $$382(ASC)] HASH:[$$390]  |PARTITIONED|
+                      -- STABLE_SORT [$$381(ASC), $$382(ASC)]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$390][$$384]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$390]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (Metadata.Synonym)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$384]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$384, $$383][$$ds_name, $$dv_name]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$384, $$383]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$ds_name, $$dv_name]  |PARTITIONED|
+                                          -- NESTED_LOOP  |UNPARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                              -- UNNEST  |UNPARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                              -- REPLICATE  |UNPARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                  -- UNNEST  |UNPARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$386]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$404][$$388]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$404]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$386][$$syn_name]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$386]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- REPLICATE  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (Metadata.Synonym)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$syn_name]  |PARTITIONED|
+                                      -- UNNEST  |UNPARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$388]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$388, $$387][$$ds_name, $$dv_name]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$384, $$383]  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (Metadata.Dataset)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$ds_name, $$dv_name]  |PARTITIONED|
+                                      -- NESTED_LOOP  |UNPARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                          -- STREAM_PROJECT  |UNPARTITIONED|
+                                            -- ASSIGN  |UNPARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                -- REPLICATE  |UNPARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                    -- UNNEST  |UNPARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                          -- UNNEST  |UNPARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/inner_right_corr.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/inner_right_corr.plan
index 37c3434..7a7a543 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/inner_right_corr.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/inner_right_corr.plan
@@ -21,7 +21,7 @@
                             -- DATASOURCE_SCAN (test.t1)  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$87]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- UNNEST  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_1.plan
new file mode 100644
index 0000000..f5d57d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_1.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$71][$$73]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (tpch.Supplier)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.Partsupp)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.Part)  |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/optimizerts/results/joins/nlj_partitioning_property_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_2.plan
new file mode 100644
index 0000000..4cd1213
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nlj_partitioning_property_2.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (tpch.Supplier)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.Partsupp)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.Part)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |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 eed20bb..f94167e 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 [$$23(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
           -- STREAM_PROJECT  |PARTITIONED|
             -- STREAM_SELECT  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+                      -- STABLE_SORT [$$31(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_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01_ps.plan
index 61cf933..9c39aef 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01_ps.plan
@@ -16,8 +16,8 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$23(ASC)]  |PARTITIONED|
+          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$24(ASC)]  |PARTITIONED|
               -- FORWARD  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- REPLICATE  |PARTITIONED|
@@ -28,7 +28,7 @@
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+                                  -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -50,7 +50,7 @@
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+                                            -- STABLE_SORT [$$31(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 80fa709..e20acdb 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 [$$25(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$26(ASC) ]  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                      -- STABLE_SORT [$$32(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_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02_ps.plan
index 159cc10..f79649b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02_ps.plan
@@ -16,8 +16,8 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$25(ASC)]  |PARTITIONED|
+          -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$26(ASC)]  |PARTITIONED|
               -- FORWARD  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- REPLICATE  |PARTITIONED|
@@ -28,7 +28,7 @@
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                  -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -50,7 +50,7 @@
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                            -- STABLE_SORT [$$32(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 eed20bb..f94167e 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 [$$23(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
           -- STREAM_PROJECT  |PARTITIONED|
             -- STREAM_SELECT  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+                      -- STABLE_SORT [$$31(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_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01_ps.plan
index 61cf933..9c39aef 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01_ps.plan
@@ -16,8 +16,8 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$23(ASC)]  |PARTITIONED|
+          -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$24(ASC)]  |PARTITIONED|
               -- FORWARD  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- REPLICATE  |PARTITIONED|
@@ -28,7 +28,7 @@
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+                                  -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -50,7 +50,7 @@
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+                                            -- STABLE_SORT [$$31(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 80fa709..e20acdb 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 [$$25(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$26(ASC) ]  |PARTITIONED|
           -- STREAM_SELECT  |PARTITIONED|
             -- STREAM_PROJECT  |PARTITIONED|
               -- ASSIGN  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                      -- STABLE_SORT [$$32(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_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02_ps.plan
index 159cc10..f79649b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02_ps.plan
@@ -16,8 +16,8 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
-            -- RANGE_PARTITION_EXCHANGE [$$25(ASC)]  |PARTITIONED|
+          -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$26(ASC)]  |PARTITIONED|
               -- FORWARD  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- REPLICATE  |PARTITIONED|
@@ -28,7 +28,7 @@
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                  -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -50,7 +50,7 @@
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- BTREE_SEARCH (tpch.Orders.Orders)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
+                                            -- STABLE_SORT [$$32(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/q01_pricing_summary_report_nt_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan
index 0128265..b5c8603 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan
@@ -22,13 +22,13 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- REPLICATE  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EXTERNAL_GROUP_BY[$$206, $$207]  |PARTITIONED|
+                      -- EXTERNAL_GROUP_BY[$$217, $$218]  |PARTITIONED|
                               {
                                 -- AGGREGATE  |LOCAL|
                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                               }
-                        -- HASH_PARTITION_EXCHANGE [$$206, $$207]  |PARTITIONED|
-                          -- EXTERNAL_GROUP_BY[$$181, $$182]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$217, $$218]  |PARTITIONED|
+                          -- EXTERNAL_GROUP_BY[$$192, $$193]  |PARTITIONED|
                                   {
                                     -- AGGREGATE  |LOCAL|
                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
@@ -51,13 +51,13 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EXTERNAL_GROUP_BY[$$206, $$207]  |PARTITIONED|
+                                -- EXTERNAL_GROUP_BY[$$217, $$218]  |PARTITIONED|
                                         {
                                           -- AGGREGATE  |LOCAL|
                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                         }
-                                  -- HASH_PARTITION_EXCHANGE [$$206, $$207]  |PARTITIONED|
-                                    -- EXTERNAL_GROUP_BY[$$181, $$182]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$217, $$218]  |PARTITIONED|
+                                    -- EXTERNAL_GROUP_BY[$$192, $$193]  |PARTITIONED|
                                             {
                                               -- AGGREGATE  |LOCAL|
                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-159-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-159-3.plan
index e932616..6c713c4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-159-3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-159-3.plan
@@ -27,7 +27,7 @@
                                                     -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- STREAM_PROJECT  |PARTITIONED|
                               -- STREAM_SELECT  |PARTITIONED|
                                 -- SUBPLAN  |PARTITIONED|
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 e0059ed..4e9ed38 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 [$$44][$$46]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$47][$$49]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- DATASOURCE_SCAN (test.s)  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$49]  |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 5d16539..3a4e8a7 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 [$$45][$$44]  |PARTITIONED|
-              -- HASH_PARTITION_EXCHANGE [$$45]  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$48][$$47]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
                 -- ASSIGN  |UNPARTITIONED|
                   -- UNNEST  |UNPARTITIONED|
                     -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_distance_static_partitioning.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_distance_static_partitioning.plan
index 25546ce..f66785b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_distance_static_partitioning.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_distance_static_partitioning.plan
@@ -7,10 +7,10 @@
             -- AGGREGATE  |PARTITIONED|
               -- STREAM_SELECT  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- SPATIAL_JOIN [$$61, $$56] [$$62, $$57]  |PARTITIONED|
+                  -- SPATIAL_JOIN [$$62, $$56] [$$63, $$57]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$61(ASC), $$56(ASC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$61]  |PARTITIONED|
+                      -- STABLE_SORT [$$62(ASC), $$56(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
                           -- UNNEST  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
@@ -21,8 +21,8 @@
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$62(ASC), $$57(ASC)]  |PARTITIONED|
-                        -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                      -- STABLE_SORT [$$63(ASC), $$57(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$63]  |PARTITIONED|
                           -- UNNEST  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2845.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2845.plan
index 4b32064..15e7546 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2845.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2845.plan
@@ -56,7 +56,7 @@
                                                                         -- DATASOURCE_SCAN (test.mds)  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- HASH_PARTITION_EXCHANGE [$$303]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- NESTED_LOOP  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_SELECT  |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 d61947a..d7bad18 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[$$173]  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$180]  |PARTITIONED|
                             {
                               -- AGGREGATE  |LOCAL|
                                 -- AGGREGATE  |LOCAL|
@@ -18,9 +18,9 @@
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- HYBRID_HASH_JOIN [$$173][$$215]  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$180][$$222]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$157]  |PARTITIONED|
+                                -- PRE_CLUSTERED_GROUP_BY[$$164]  |PARTITIONED|
                                         {
                                           -- AGGREGATE  |LOCAL|
                                             -- AGGREGATE  |LOCAL|
@@ -28,13 +28,13 @@
                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                                         }
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$157(ASC)]  |PARTITIONED|
+                                    -- STABLE_SORT [$$164(ASC)]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$157][$$171]  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$164][$$178]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- PRE_CLUSTERED_GROUP_BY[$$137]  |PARTITIONED|
+                                                -- PRE_CLUSTERED_GROUP_BY[$$144]  |PARTITIONED|
                                                         {
                                                           -- AGGREGATE  |LOCAL|
                                                             -- AGGREGATE  |LOCAL|
@@ -44,7 +44,7 @@
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- HYBRID_HASH_JOIN [$$137][$$147]  |PARTITIONED|
+                                                        -- HYBRID_HASH_JOIN [$$144][$$154]  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                               -- ASSIGN  |PARTITIONED|
@@ -60,7 +60,7 @@
                                                               -- ASSIGN  |PARTITIONED|
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                   -- REPLICATE  |PARTITIONED|
-                                                                    -- HASH_PARTITION_EXCHANGE [$$198]  |PARTITIONED|
+                                                                    -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
                                                                       -- STREAM_PROJECT  |PARTITIONED|
                                                                         -- ASSIGN  |PARTITIONED|
                                                                           -- STREAM_PROJECT  |PARTITIONED|
@@ -73,35 +73,35 @@
                                                                                         -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                              -- HASH_PARTITION_EXCHANGE [$$171]  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$178]  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- SORT_GROUP_BY[$$220, $$221]  |PARTITIONED|
+                                                        -- SORT_GROUP_BY[$$227, $$228]  |PARTITIONED|
                                                                 {
                                                                   -- AGGREGATE  |LOCAL|
                                                                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                                 }
-                                                          -- HASH_PARTITION_EXCHANGE [$$220, $$221]  |PARTITIONED|
-                                                            -- PRE_CLUSTERED_GROUP_BY[$$158, $$161]  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$227, $$228]  |PARTITIONED|
+                                                            -- PRE_CLUSTERED_GROUP_BY[$$165, $$168]  |PARTITIONED|
                                                                     {
                                                                       -- AGGREGATE  |LOCAL|
                                                                         -- STREAM_SELECT  |LOCAL|
                                                                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                                     }
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STABLE_SORT [$$158(ASC), $$161(ASC)]  |PARTITIONED|
+                                                                -- STABLE_SORT [$$165(ASC), $$168(ASC)]  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        -- HYBRID_HASH_JOIN [$$150][$$149]  |PARTITIONED|
+                                                                        -- HYBRID_HASH_JOIN [$$157][$$156]  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                                               -- ASSIGN  |PARTITIONED|
                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                   -- REPLICATE  |PARTITIONED|
-                                                                                    -- HASH_PARTITION_EXCHANGE [$$196]  |PARTITIONED|
+                                                                                    -- HASH_PARTITION_EXCHANGE [$$204]  |PARTITIONED|
                                                                                       -- RUNNING_AGGREGATE  |PARTITIONED|
                                                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                                                           -- ASSIGN  |PARTITIONED|
@@ -110,7 +110,7 @@
                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                   -- REPLICATE  |PARTITIONED|
                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$199]  |PARTITIONED|
+                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$206]  |PARTITIONED|
                                                                                                               {
                                                                                                                 -- AGGREGATE  |LOCAL|
                                                                                                                   -- AGGREGATE  |LOCAL|
@@ -120,7 +120,7 @@
                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                           -- STREAM_PROJECT  |PARTITIONED|
                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              -- HYBRID_HASH_JOIN [$$199][$$200]  |PARTITIONED|
+                                                                                                              -- HYBRID_HASH_JOIN [$$206][$$207]  |PARTITIONED|
                                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                   -- REPLICATE  |PARTITIONED|
                                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -133,7 +133,7 @@
                                                                                                                     -- ASSIGN  |PARTITIONED|
                                                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                         -- REPLICATE  |PARTITIONED|
-                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$198]  |PARTITIONED|
+                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
                                                                                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                               -- ASSIGN  |PARTITIONED|
                                                                                                                                 -- STREAM_PROJECT  |PARTITIONED|
@@ -146,7 +146,7 @@
                                                                                                                                               -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
                                                                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                          -- HASH_PARTITION_EXCHANGE [$$149]  |PARTITIONED|
+                                                                          -- HASH_PARTITION_EXCHANGE [$$156]  |PARTITIONED|
                                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                                               -- ASSIGN  |PARTITIONED|
                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -157,30 +157,30 @@
                                                                                           -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$215]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$222]  |PARTITIONED|
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   -- ASSIGN  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- SORT_GROUP_BY[$$226, $$227]  |PARTITIONED|
+                                        -- SORT_GROUP_BY[$$233, $$234]  |PARTITIONED|
                                                 {
                                                   -- AGGREGATE  |LOCAL|
                                                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                 }
-                                          -- HASH_PARTITION_EXCHANGE [$$226, $$227]  |PARTITIONED|
-                                            -- PRE_CLUSTERED_GROUP_BY[$$174, $$177]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$233, $$234]  |PARTITIONED|
+                                            -- PRE_CLUSTERED_GROUP_BY[$$181, $$184]  |PARTITIONED|
                                                     {
                                                       -- AGGREGATE  |LOCAL|
                                                         -- STREAM_SELECT  |LOCAL|
                                                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                     }
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- STABLE_SORT [$$174(ASC), $$177(ASC)]  |PARTITIONED|
+                                                -- STABLE_SORT [$$181(ASC), $$184(ASC)]  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- HYBRID_HASH_JOIN [$$152][$$151]  |PARTITIONED|
-                                                          -- HASH_PARTITION_EXCHANGE [$$152]  |PARTITIONED|
+                                                        -- HYBRID_HASH_JOIN [$$159][$$158]  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$159]  |PARTITIONED|
                                                             -- RUNNING_AGGREGATE  |PARTITIONED|
                                                               -- STREAM_PROJECT  |PARTITIONED|
                                                                 -- ASSIGN  |PARTITIONED|
@@ -189,7 +189,7 @@
                                                                       -- STREAM_PROJECT  |PARTITIONED|
                                                                         -- ASSIGN  |PARTITIONED|
                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- PRE_CLUSTERED_GROUP_BY[$$178]  |PARTITIONED|
+                                                                            -- PRE_CLUSTERED_GROUP_BY[$$185]  |PARTITIONED|
                                                                                     {
                                                                                       -- AGGREGATE  |LOCAL|
                                                                                         -- AGGREGATE  |LOCAL|
@@ -197,18 +197,18 @@
                                                                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                                                     }
                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                -- STABLE_SORT [$$178(ASC)]  |PARTITIONED|
+                                                                                -- STABLE_SORT [$$185(ASC)]  |PARTITIONED|
                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        -- HYBRID_HASH_JOIN [$$178][$$181]  |PARTITIONED|
+                                                                                        -- HYBRID_HASH_JOIN [$$185][$$188]  |PARTITIONED|
                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                                                               -- ASSIGN  |PARTITIONED|
                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                   -- REPLICATE  |PARTITIONED|
                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$199]  |PARTITIONED|
+                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$206]  |PARTITIONED|
                                                                                                               {
                                                                                                                 -- AGGREGATE  |LOCAL|
                                                                                                                   -- AGGREGATE  |LOCAL|
@@ -218,7 +218,7 @@
                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                           -- STREAM_PROJECT  |PARTITIONED|
                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              -- HYBRID_HASH_JOIN [$$199][$$200]  |PARTITIONED|
+                                                                                                              -- HYBRID_HASH_JOIN [$$206][$$207]  |PARTITIONED|
                                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                   -- REPLICATE  |PARTITIONED|
                                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -231,7 +231,7 @@
                                                                                                                     -- ASSIGN  |PARTITIONED|
                                                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                         -- REPLICATE  |PARTITIONED|
-                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$198]  |PARTITIONED|
+                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
                                                                                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                               -- ASSIGN  |PARTITIONED|
                                                                                                                                 -- STREAM_PROJECT  |PARTITIONED|
@@ -244,31 +244,31 @@
                                                                                                                                               -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
                                                                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                          -- HASH_PARTITION_EXCHANGE [$$181]  |PARTITIONED|
+                                                                                          -- HASH_PARTITION_EXCHANGE [$$188]  |PARTITIONED|
                                                                                             -- ASSIGN  |PARTITIONED|
                                                                                               -- STREAM_PROJECT  |PARTITIONED|
                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  -- SORT_GROUP_BY[$$223, $$224]  |PARTITIONED|
+                                                                                                  -- SORT_GROUP_BY[$$230, $$231]  |PARTITIONED|
                                                                                                           {
                                                                                                             -- AGGREGATE  |LOCAL|
                                                                                                               -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                                                                           }
-                                                                                                    -- HASH_PARTITION_EXCHANGE [$$223, $$224]  |PARTITIONED|
-                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$193, $$194]  |PARTITIONED|
+                                                                                                    -- HASH_PARTITION_EXCHANGE [$$230, $$231]  |PARTITIONED|
+                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$200, $$201]  |PARTITIONED|
                                                                                                               {
                                                                                                                 -- AGGREGATE  |LOCAL|
                                                                                                                   -- STREAM_SELECT  |LOCAL|
                                                                                                                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                                                                               }
                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                          -- STABLE_SORT [$$193(ASC), $$194(ASC)]  |PARTITIONED|
+                                                                                                          -- STABLE_SORT [$$200(ASC), $$201(ASC)]  |PARTITIONED|
                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                               -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- HYBRID_HASH_JOIN [$$196][$$198]  |PARTITIONED|
+                                                                                                                  -- HYBRID_HASH_JOIN [$$204][$$205]  |PARTITIONED|
                                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                       -- REPLICATE  |PARTITIONED|
-                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$196]  |PARTITIONED|
+                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$204]  |PARTITIONED|
                                                                                                                           -- RUNNING_AGGREGATE  |PARTITIONED|
                                                                                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                               -- ASSIGN  |PARTITIONED|
@@ -277,7 +277,7 @@
                                                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                       -- REPLICATE  |PARTITIONED|
                                                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                          -- PRE_CLUSTERED_GROUP_BY[$$199]  |PARTITIONED|
+                                                                                                                                          -- PRE_CLUSTERED_GROUP_BY[$$206]  |PARTITIONED|
                                                                                                                                                   {
                                                                                                                                                     -- AGGREGATE  |LOCAL|
                                                                                                                                                       -- AGGREGATE  |LOCAL|
@@ -287,7 +287,7 @@
                                                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                               -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                                  -- HYBRID_HASH_JOIN [$$199][$$200]  |PARTITIONED|
+                                                                                                                                                  -- HYBRID_HASH_JOIN [$$206][$$207]  |PARTITIONED|
                                                                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                       -- REPLICATE  |PARTITIONED|
                                                                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -300,7 +300,7 @@
                                                                                                                                                         -- ASSIGN  |PARTITIONED|
                                                                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                             -- REPLICATE  |PARTITIONED|
-                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$198]  |PARTITIONED|
+                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
                                                                                                                                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                                                                   -- ASSIGN  |PARTITIONED|
                                                                                                                                                                     -- STREAM_PROJECT  |PARTITIONED|
@@ -315,7 +315,7 @@
                                                                                                                                                                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                       -- REPLICATE  |PARTITIONED|
-                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$198]  |PARTITIONED|
+                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$205]  |PARTITIONED|
                                                                                                                           -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                             -- ASSIGN  |PARTITIONED|
                                                                                                                               -- STREAM_PROJECT  |PARTITIONED|
@@ -328,7 +328,7 @@
                                                                                                                                             -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
                                                                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                          -- HASH_PARTITION_EXCHANGE [$$151]  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$158]  |PARTITIONED|
                                                             -- STREAM_PROJECT  |PARTITIONED|
                                                               -- ASSIGN  |PARTITIONED|
                                                                 -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.10.plan
new file mode 100644
index 0000000..9ea1780
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.10.plan
@@ -0,0 +1,35 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                        -- MICRO_STABLE_SORT [$$67(ASC)]  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                  -- NESTED_LOOP  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- UNNEST  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.11.plan
new file mode 100644
index 0000000..ebfcfc2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.11.plan
@@ -0,0 +1,43 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                        -- MICRO_STABLE_SORT [$$67(ASC)]  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                        -- MICRO_STABLE_SORT [$$72(ASC)]  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                  -- NESTED_LOOP  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- UNNEST  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.12.plan
new file mode 100644
index 0000000..a8f346f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.12.plan
@@ -0,0 +1,39 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                        -- MICRO_STABLE_SORT [$$71(ASC)]  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                  -- NESTED_LOOP  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- UNNEST  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.13.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.13.plan
new file mode 100644
index 0000000..439b1d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.13.plan
@@ -0,0 +1,57 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                        -- MICRO_STABLE_SORT [$$82(ASC)]  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                        -- MICRO_STABLE_SORT [$$92(ASC)]  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                  -- NESTED_LOOP  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- NESTED_LOOP  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                              -- UNNEST  |UNPARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.14.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.14.plan
new file mode 100644
index 0000000..c57b6f9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.14.plan
@@ -0,0 +1,63 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                        -- MICRO_STABLE_SORT [$$106(ASC)]  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                        -- MICRO_STABLE_SORT [$$116(ASC)]  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                        -- MICRO_STABLE_SORT [$$111(ASC)]  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                  -- NESTED_LOOP  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- NESTED_LOOP  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                              -- UNNEST  |UNPARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.d3)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.9.plan
new file mode 100644
index 0000000..8802ff5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/aggregate-sql-sugar/distinct_mixed/distinct_mixed.9.plan
@@ -0,0 +1,35 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$g(ASC) ]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$g]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- MICRO_PRE_SORTED_DISTINCT_BY  |LOCAL|
+                        -- MICRO_STABLE_SORT [$$63(ASC)]  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$g(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$g]  |PARTITIONED|
+                  -- NESTED_LOOP  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (test.d1)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- UNNEST  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.d2)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-3.plan
new file mode 100644
index 0000000..bd6d8cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-3.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$37(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.tenk2.tenk2)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH (test.tenk2.idx_tenk2_1k)  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-5.plan
new file mode 100644
index 0000000..bd6d8cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-5.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$37(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.tenk2.tenk2)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH (test.tenk2.idx_tenk2_1k)  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-6.plan
new file mode 100644
index 0000000..b6b4ba0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-6.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$37(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$35][$$34]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$35]  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (test.tenk2)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-7.plan
new file mode 100644
index 0000000..bd6d8cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-join/hints-indexnl-params/hints-indexnl-params-7.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$37(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$37(ASC)]  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (test.tenk2.tenk2)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$42(ASC)]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH (test.tenk2.idx_tenk2_1k)  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (test.tenk1.tenk1)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-1.plan
new file mode 100644
index 0000000..61695a3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-1.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- HYBRID_HASH_JOIN [$$20][$$21]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- DATASOURCE_SCAN (test.TestSet)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- BROADCAST_EXCHANGE  |PARTITIONED|
+            -- UNNEST  |UNPARTITIONED|
+              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-2.plan
new file mode 100644
index 0000000..318f7d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/disjunctive-predicate/disjunctive-predicate-2.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- HYBRID_HASH_JOIN [$$19][$$21]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN (test.TestSet)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+          -- BROADCAST_EXCHANGE  |PARTITIONED|
+            -- UNNEST  |UNPARTITIONED|
+              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-11.plan
new file mode 100644
index 0000000..a2a9642
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-11.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$20][$$22]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-12.plan
new file mode 100644
index 0000000..a2a9642
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/btree-index-selection/hints-skip-index/hints-skip-index-12.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$20][$$22]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q2.plan
new file mode 100644
index 0000000..5090c1a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q2.plan
@@ -0,0 +1,162 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$260(ASC), $$261(ASC), $$226(ASC) ]  |PARTITIONED|
+            -- STREAM_LIMIT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [topK: 100] [$$260(ASC), $$261(ASC), $$226(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$238][$$239]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$238]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$245][$$246]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$245]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$251][$$228]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$251]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$226, $$240][$$227, $$249]  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$226, $$240]  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- HYBRID_HASH_JOIN [$$s_i_id][$$226]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- SORT_GROUP_BY[$$271]  |PARTITIONED|
+                                                                        {
+                                                                          -- AGGREGATE  |LOCAL|
+                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                        }
+                                                                  -- HASH_PARTITION_EXCHANGE [$$271]  |PARTITIONED|
+                                                                    -- SORT_GROUP_BY[$$225]  |PARTITIONED|
+                                                                            {
+                                                                              -- AGGREGATE  |LOCAL|
+                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                            }
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- HYBRID_HASH_JOIN [$$256][$$247]  |PARTITIONED|
+                                                                              -- HASH_PARTITION_EXCHANGE [$$256]  |PARTITIONED|
+                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- HYBRID_HASH_JOIN [$$255][$$243]  |PARTITIONED|
+                                                                                      -- HASH_PARTITION_EXCHANGE [$$255]  |PARTITIONED|
+                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- HYBRID_HASH_JOIN [$$241][$$242]  |PARTITIONED|
+                                                                                              -- HASH_PARTITION_EXCHANGE [$$241]  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                  -- HASH_PARTITION_EXCHANGE [$$242]  |PARTITIONED|
+                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                      -- HASH_PARTITION_EXCHANGE [$$243]  |PARTITIONED|
+                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                              -- HASH_PARTITION_EXCHANGE [$$247]  |PARTITIONED|
+                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$226]  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$227, $$249]  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- REPLICATE  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$228]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$246]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- REPLICATE  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- REPLICATE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$242]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q21.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q21.plan
new file mode 100644
index 0000000..dd60dcd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q21.plan
@@ -0,0 +1,116 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$su_name(ASC) ]  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_LIMIT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- SORT_GROUP_BY[$$378]  |PARTITIONED|
+                          {
+                            -- AGGREGATE  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                          }
+                    -- HASH_PARTITION_EXCHANGE [$$378]  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$su_name]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- SORT_GROUP_BY[$$369, $$370, $$371, $$372, $$373, $$374, $$375, $$376]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- HASH_PARTITION_EXCHANGE [$$369, $$370, $$371, $$372, $$373, $$374, $$375, $$376]  |PARTITIONED|
+                                      -- SORT_GROUP_BY[$$331, $$357, $$356, $$333, $$334, $$330, $$329, $$358]  |PARTITIONED|
+                                              {
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                              }
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- NESTED_LOOP  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- HYBRID_HASH_JOIN [$$344][$$333]  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$344]  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- HYBRID_HASH_JOIN [$$345][$$334]  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$345]  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- HYBRID_HASH_JOIN [$$330, $$329][$$331, $$341]  |PARTITIONED|
+                                                                        -- HASH_PARTITION_EXCHANGE [$$330, $$329]  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                        -- HASH_PARTITION_EXCHANGE [$$331, $$341]  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- UNNEST  |PARTITIONED|
+                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$334]  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$333]  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- UNNEST  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- REPLICATE  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q5.plan
new file mode 100644
index 0000000..165db4cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q5.plan
@@ -0,0 +1,98 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$#2(DESC) ]  |PARTITIONED|
+          -- STABLE_SORT [$#2(DESC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$278]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- HASH_PARTITION_EXCHANGE [$$278]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$261]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$260, $$276][$$252, $$255]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$260, $$276]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$240, $$267][$$231, $$232]  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$240, $$267]  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$262, $$263, $$264][$$246, $$240, $$249]  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$262, $$263, $$264]  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- HYBRID_HASH_JOIN [$$260][$$244]  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$260]  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- HYBRID_HASH_JOIN [$$242][$$243]  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$242]  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$243]  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$244]  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$246, $$240, $$249]  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- UNNEST  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$231, $$232]  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$252, $$255]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q7.plan
new file mode 100644
index 0000000..f50b5a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q7.plan
@@ -0,0 +1,104 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$su_nationkey(ASC), $#1(ASC), $#2(ASC) ]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$311, $$312, $$313]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- HASH_PARTITION_EXCHANGE [$$311, $$312, $$313]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$262, $$258, $$259]  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$262][$$288]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$262]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$309][$$285]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$309]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- HYBRID_HASH_JOIN [$$295, $$296][$$260, $$261]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$295, $$296]  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- HYBRID_HASH_JOIN [$$290, $$291, $$292][$$276, $$278, $$280]  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$290, $$291, $$292]  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- HYBRID_HASH_JOIN [$$289][$$273]  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- REPLICATE  |PARTITIONED|
+                                                                        -- HASH_PARTITION_EXCHANGE [$$289]  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                    -- HASH_PARTITION_EXCHANGE [$$273]  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$276, $$278, $$280]  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- UNNEST  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$260, $$261]  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$285]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$289]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q8.plan
new file mode 100644
index 0000000..34a9df6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q8.plan
@@ -0,0 +1,128 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$#1(ASC) ]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$334]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- HASH_PARTITION_EXCHANGE [$$334]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$278]  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$309][$$310]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$309]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$331][$$325]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$331]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$290, $$320][$$280, $$279]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$290, $$320]  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- HYBRID_HASH_JOIN [$$292][$$293]  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$292]  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- HYBRID_HASH_JOIN [$$295][$$315]  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$295]  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- HYBRID_HASH_JOIN [$$297, $$299, $$301][$$317, $$318, $$316]  |PARTITIONED|
+                                                                        -- HASH_PARTITION_EXCHANGE [$$297, $$299, $$301]  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- HYBRID_HASH_JOIN [$$290][$$308]  |PARTITIONED|
+                                                                                -- HASH_PARTITION_EXCHANGE [$$290]  |PARTITIONED|
+                                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                          -- UNNEST  |PARTITIONED|
+                                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                -- HASH_PARTITION_EXCHANGE [$$308]  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                        -- HASH_PARTITION_EXCHANGE [$$317, $$318, $$316]  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- DATASOURCE_SCAN (test.customer)  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$315]  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- REPLICATE  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$293]  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (test.region)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$280, $$279]  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$325]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$310]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- REPLICATE  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q9.plan
new file mode 100644
index 0000000..5557daa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/ch2/ch2_q9.plan
@@ -0,0 +1,81 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$n_name(ASC), $#1(DESC) ]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$218, $$219]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- HASH_PARTITION_EXCHANGE [$$218, $$219]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$213, $$186]  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$201][$$202]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$201]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$216][$$212]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$216]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$194, $$205][$$188, $$187]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- HYBRID_HASH_JOIN [$$194][$$195]  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$194]  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- UNNEST  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- DATASOURCE_SCAN (test.orders)  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$195]  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (test.item)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$188]  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN (test.stock)  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.supplier)  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$202]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.nation)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/collocated.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/collocated.plan
new file mode 100644
index 0000000..0f4f7a3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/collocated.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (colocated.Visitors.Visitors)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (colocated.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/denorm-cust-order.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/denorm-cust-order.plan
new file mode 100644
index 0000000..0133d09
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/denorm-cust-order.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$93]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH (custorder.Customers.Customers)  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$95(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$95]  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (custorder.Orders)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/extract-common-operators/extract-common-operators.01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/extract-common-operators/extract-common-operators.01.plan
new file mode 100644
index 0000000..d7243b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/extract-common-operators/extract-common-operators.01.plan
@@ -0,0 +1,90 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$381(ASC), $$382(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$381(ASC), $$382(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$390][$$386]  |PARTITIONED|
+                    -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$381(ASC), $$382(ASC)] HASH:[$$390]  |PARTITIONED|
+                      -- STABLE_SORT [$$381(ASC), $$382(ASC)]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$390][$$384]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$390]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (Metadata.Synonym)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$384]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- HYBRID_HASH_JOIN [$$384][$$ds_name]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$384]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH (Metadata.Dataset.Dataset)  |PARTITIONED|
+                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                      -- UNNEST  |UNPARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$ds_name]  |PARTITIONED|
+                                              -- UNNEST  |UNPARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$386]  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- HYBRID_HASH_JOIN [$$404][$$388]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$404]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$386][$$syn_name]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$386]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- REPLICATE  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (Metadata.Synonym)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$syn_name]  |PARTITIONED|
+                                      -- UNNEST  |UNPARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$388]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$388][$$ds_name]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$384]  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (Metadata.Dataset.Dataset)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- UNNEST  |UNPARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$ds_name]  |PARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/filter_on_meta_with_idx_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/filter_on_meta_with_idx_2.plan
new file mode 100644
index 0000000..e38f6f9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/filter_on_meta_with_idx_2.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.KVStore.KVStore)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.KVStore.s_location)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (test.OfficerLocations)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/group-by/listify-3.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/group-by/listify-3.1.plan
new file mode 100644
index 0000000..ec40853
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/group-by/listify-3.1.plan
@@ -0,0 +1,43 @@
+-- DISTRIBUTE_RESULT  |LOCAL|
+  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+    -- STREAM_PROJECT  |LOCAL|
+      -- ASSIGN  |LOCAL|
+        -- AGGREGATE  |LOCAL|
+          -- AGGREGATE  |LOCAL|
+            -- STREAM_PROJECT  |LOCAL|
+              -- ASSIGN  |LOCAL|
+                -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                  -- NESTED_LOOP  |LOCAL|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- UNNEST  |UNPARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                      -- NESTED_LOOP  |LOCAL|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STREAM_PROJECT  |LOCAL|
+                            -- ASSIGN  |LOCAL|
+                              -- STREAM_PROJECT  |LOCAL|
+                                -- ASSIGN  |LOCAL|
+                                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                                    -- REPLICATE  |LOCAL|
+                                      -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_PROJECT  |LOCAL|
+                                            -- ASSIGN  |LOCAL|
+                                              -- AGGREGATE  |LOCAL|
+                                                -- AGGREGATE  |LOCAL|
+                                                  -- UNNEST  |UNPARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                          -- STREAM_PROJECT  |LOCAL|
+                            -- ASSIGN  |LOCAL|
+                              -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                                -- REPLICATE  |LOCAL|
+                                  -- ONE_TO_ONE_EXCHANGE  |LOCAL|
+                                    -- AGGREGATE  |LOCAL|
+                                      -- STREAM_PROJECT  |LOCAL|
+                                        -- ASSIGN  |LOCAL|
+                                          -- AGGREGATE  |LOCAL|
+                                            -- AGGREGATE  |LOCAL|
+                                              -- UNNEST  |UNPARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_1.plan
new file mode 100644
index 0000000..0585072
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_1.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$73][$$71]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (tpch.Partsupp)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN (tpch.Supplier)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (tpch.Part)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_2.plan
new file mode 100644
index 0000000..4367a33
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/joins/nlj_partitioning_property_2.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.Partsupp)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.Part)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN (tpch.Supplier)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/leftouterjoin/query-ASTERIXDB-2857.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/leftouterjoin/query-ASTERIXDB-2857.plan
new file mode 100644
index 0000000..6ecdc2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/leftouterjoin/query-ASTERIXDB-2857.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$136(ASC), $$137(ASC), $#3(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$136(ASC), $$137(ASC), $#3(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- NESTED_LOOP  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.tenk.tenk)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/meta/indexes_on_dataset_with_meta_05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/meta/indexes_on_dataset_with_meta_05.plan
new file mode 100644
index 0000000..6013ad8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/meta/indexes_on_dataset_with_meta_05.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$29(ASC) ]  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (test.DS2.DS2)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.DS1)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-index/btree-index-join/ASTERIXDB-2199.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-index/btree-index-join/ASTERIXDB-2199.plan
new file mode 100644
index 0000000..c091a3b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-index/btree-index-join/ASTERIXDB-2199.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (Facebook.Friendship.Friendship)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (Facebook.Friendship)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan
new file mode 100644
index 0000000..4eb7b4f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/05.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$46(ASC), $$47(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$46(ASC), $$47(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$54(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TestOpen2.idx_t2_s)  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.plan
new file mode 100644
index 0000000..e2f07be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.plan
@@ -0,0 +1,27 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$38(ASC), $$39(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (test.TestOpen2.idx_t2_s)  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.plan
new file mode 100644
index 0000000..af04b9c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.plan
@@ -0,0 +1,58 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$38(ASC), $$39(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (test.TestOpen2.idx_t2_s)  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- REPLICATE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.TestOpen2.TestOpen2)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$43(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (test.TestOpen2.idx_t2_s)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (test.TestOpen1)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q08_group_by.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q08_group_by.plan
new file mode 100644
index 0000000..fbaefe2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q08_group_by.plan
@@ -0,0 +1,55 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (q08_group_by.Supplier.Supplier)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$221(ASC)]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$221]  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (q08_group_by.Part.Part)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$218(ASC)]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$218]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (q08_group_by.LineItem.LineItem)  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH (q08_group_by.Region.Region)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$204(ASC)]  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$204]  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH (q08_group_by.Nation.Nation)  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- STABLE_SORT [$$206(ASC)]  |PARTITIONED|
+                                                                              -- HASH_PARTITION_EXCHANGE [$$206]  |PARTITIONED|
+                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- BTREE_SEARCH (q08_group_by.Customer.Customer)  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- STABLE_SORT [$$209(ASC)]  |PARTITIONED|
+                                                                                              -- HASH_PARTITION_EXCHANGE [$$209]  |PARTITIONED|
+                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- DATASOURCE_SCAN (q08_group_by.Orders)  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q09_group_by.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q09_group_by.plan
new file mode 100644
index 0000000..9a390a3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/q09_group_by.plan
@@ -0,0 +1,46 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (q09_group_by.Part.Part)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$161(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$161]  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$161, $$146][$$140, $$141]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$161, $$146]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (q09_group_by.Nation.Nation)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$149(ASC)]  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$149]  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH (q09_group_by.Supplier.Supplier)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$146(ASC)]  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$146]  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- DATASOURCE_SCAN (q09_group_by.LineItem)  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (q09_group_by.Partsupp)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue849.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue849.plan
new file mode 100644
index 0000000..59b09a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue849.plan
@@ -0,0 +1,29 @@
+-- COMMIT  |PARTITIONED|
+  -- STREAM_PROJECT  |PARTITIONED|
+    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+      -- INSERT_DELETE  |PARTITIONED|
+        -- HASH_PARTITION_EXCHANGE [$$3]  |PARTITIONED|
+          -- ASSIGN  |UNPARTITIONED|
+            -- ASSIGN  |UNPARTITIONED|
+              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+-- COMMIT  |PARTITIONED|
+  -- STREAM_PROJECT  |PARTITIONED|
+    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+      -- INSERT_DELETE  |PARTITIONED|
+        -- HASH_PARTITION_EXCHANGE [$$3]  |PARTITIONED|
+          -- ASSIGN  |UNPARTITIONED|
+            -- ASSIGN  |UNPARTITIONED|
+              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH (test.s.s)  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
+                  -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
+                    -- ASSIGN  |UNPARTITIONED|
+                      -- UNNEST  |UNPARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization.plan
new file mode 100644
index 0000000..34f5238
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization.plan
@@ -0,0 +1,20 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$91(ASC), $$92(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$91(ASC), $$92(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TinySocial.FacebookUsers.FacebookUsers)  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization_ps.plan
new file mode 100644
index 0000000..2881c66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/split-materialization_ps.plan
@@ -0,0 +1,45 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$91(ASC), $$92(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$91(ASC), $$92(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH (TinySocial.FacebookUsers.FacebookUsers)  |PARTITIONED|
+                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH (TinySocial.FacebookUsers.FacebookUsers)  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (TinySocial.FacebookUsers)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1580.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1580.plan
new file mode 100644
index 0000000..111de48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1580.plan
@@ -0,0 +1,59 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$128(ASC) ]  |PARTITIONED|
+            -- STREAM_LIMIT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STABLE_SORT [topK: 100] [$$128(ASC)]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$137]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- HASH_PARTITION_EXCHANGE [$$137]  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$118]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (tpcds.item.item)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$121(ASC)]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$121]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH (tpcds.date_dim.date_dim)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STABLE_SORT [$$129(ASC)]  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- BTREE_SEARCH (tpcds.customer_address.customer_address)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- STABLE_SORT [$$132(ASC)]  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- BTREE_SEARCH (tpcds.customer.customer)  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- STABLE_SORT [$$134(ASC)]  |PARTITIONED|
+                                                                                      -- HASH_PARTITION_EXCHANGE [$$134]  |PARTITIONED|
+                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1591.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1591.plan
new file mode 100644
index 0000000..95b99e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpcds/query-ASTERIXDB-1591.plan
@@ -0,0 +1,128 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- SORT_MERGE_EXCHANGE [$$184(ASC) ]  |PARTITIONED|
+              -- STREAM_LIMIT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$193, $$194]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- HASH_PARTITION_EXCHANGE [$$193, $$194]  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$181, $$182]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$181(ASC), $$182(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$181][$$163]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$181]  |PARTITIONED|
+                                            -- SORT_GROUP_BY[$$190, $$191]  |PARTITIONED|
+                                                    {
+                                                      -- AGGREGATE  |LOCAL|
+                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                    }
+                                              -- HASH_PARTITION_EXCHANGE [$$190, $$191]  |PARTITIONED|
+                                                -- PRE_CLUSTERED_GROUP_BY[$$178, $$179]  |PARTITIONED|
+                                                        {
+                                                          -- AGGREGATE  |LOCAL|
+                                                            -- STREAM_SELECT  |LOCAL|
+                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                        }
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STABLE_SORT [$$178(ASC), $$179(ASC)]  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- HYBRID_HASH_JOIN [$$178][$$161]  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$178]  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- SORT_GROUP_BY[$$187, $$188]  |PARTITIONED|
+                                                                              {
+                                                                                -- AGGREGATE  |LOCAL|
+                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                              }
+                                                                        -- HASH_PARTITION_EXCHANGE [$$187, $$188]  |PARTITIONED|
+                                                                          -- PRE_CLUSTERED_GROUP_BY[$$142, $$143]  |PARTITIONED|
+                                                                                  {
+                                                                                    -- AGGREGATE  |LOCAL|
+                                                                                      -- STREAM_SELECT  |LOCAL|
+                                                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                  }
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- STABLE_SORT [$$142(ASC), $$143(ASC)]  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- HYBRID_HASH_JOIN [$$142][$$159]  |PARTITIONED|
+                                                                                        -- HASH_PARTITION_EXCHANGE [$$142]  |PARTITIONED|
+                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- BTREE_SEARCH (tpcds.customer_address.customer_address)  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- STABLE_SORT [$$153(ASC)]  |PARTITIONED|
+                                                                                                    -- HASH_PARTITION_EXCHANGE [$$153]  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- DATASOURCE_SCAN (tpcds.customer)  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                        -- HASH_PARTITION_EXCHANGE [$$159]  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- BTREE_SEARCH (tpcds.date_dim.date_dim)  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- STABLE_SORT [$$164(ASC)]  |PARTITIONED|
+                                                                                                        -- HASH_PARTITION_EXCHANGE [$$164]  |PARTITIONED|
+                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- DATASOURCE_SCAN (tpcds.store_sales)  |PARTITIONED|
+                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$161]  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH (tpcds.date_dim.date_dim)  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- STABLE_SORT [$$166(ASC)]  |PARTITIONED|
+                                                                              -- HASH_PARTITION_EXCHANGE [$$166]  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- DATASOURCE_SCAN (tpcds.web_sales)  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$163]  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH (tpcds.date_dim.date_dim)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STABLE_SORT [$$168(ASC)]  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$168]  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN (tpcds.catalog_sales)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping.plan
new file mode 100644
index 0000000..7a08146
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping.plan
@@ -0,0 +1,43 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$131]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$114]  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$120][$$114]  |PARTITIONED|
+                        -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                          -- UNNEST  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$122][$$118]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast.plan
new file mode 100644
index 0000000..613c7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast.plan
@@ -0,0 +1,43 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$131]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$114]  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$120][$$114]  |PARTITIONED|
+                        -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                          -- UNNEST  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$121][$$118]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$121]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast_ps.plan
new file mode 100644
index 0000000..7fce16c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_broadcast_ps.plan
@@ -0,0 +1,95 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$l_shipmode(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$l_shipmode(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$131]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$114]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$120][$$114]  |PARTITIONED|
+                                    -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                                      -- UNNEST  |UNPARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$121][$$118]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$121]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$131]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$114]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$120][$$114]  |PARTITIONED|
+                                              -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$121][$$118]  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$121]  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_ps.plan
new file mode 100644
index 0000000..0639aa6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping_ps.plan
@@ -0,0 +1,95 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$l_shipmode(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$l_shipmode(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$131]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$114]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$120][$$114]  |PARTITIONED|
+                                    -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                                      -- UNNEST  |UNPARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$122][$$118]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                  -- AGGREGATE  |UNPARTITIONED|
+                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                      -- AGGREGATE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- REPLICATE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$131]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$114]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$120][$$114]  |PARTITIONED|
+                                              -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$122][$$118]  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN (tpch.Orders)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/only_batch_lookup.xml b/asterixdb/asterix-app/src/test/resources/runtimets/only_batch_lookup.xml
new file mode 100644
index 0000000..334dd52
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/only_batch_lookup.xml
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT 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-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+  <test-group name="failed">
+  </test-group>
+</test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.1.ddl.sqlpp
new file mode 100644
index 0000000..c0aadca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.1.ddl.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.
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+use test1;
+
+create type t1 as closed {
+  c1 : bigint,
+  c2 : bigint
+};
+
+create dataset ds1(t1) primary key c1;
+
+create index idx1 on ds1(c2);
+
+create view v1 as select value v from ds1 v;
+
+create synonym s1 for v1;
+
+create function f1() {
+  select "Dataverse" as k, DataverseName as dv, DataverseName as n from Metadata.`Dataverse`
+  union all
+  select "Datatype" as k, DataverseName as dv, DatatypeName as n from Metadata.`Datatype`
+  union all
+  select "Dataset" as k, DataverseName as dv, DatasetName as n from Metadata.`Dataset`
+  union all
+  select "Index" as k, DataverseName as dv, IndexName as n from Metadata.`Index` where not(isPrimary)
+  union all
+  select "Synonym" as k, DataverseName as dv, SynonymName as n from Metadata.`Synonym`
+  union all
+  select "Function" as k, DataverseName as dv, Name as n from Metadata.`Function`
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.10.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.10.query.sqlpp
index 6095b26..53097a4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.10.query.sqlpp
@@ -17,8 +17,10 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Check that objects were not created
+ */
 
-select value v from range(1,2) v where v > ?;
+select count(*) cnt
+from test1.f1() t
+where dv like "test2";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.2.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.2.query.sqlpp
index 6095b26..1e3fb34 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.2.query.sqlpp
@@ -17,8 +17,7 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
+select k, n
+from test1.f1() t
+where dv like "test1"
+order by k, n;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.3.ddl.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.3.ddl.sqlpp
index 6095b26..b913179 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.3.ddl.sqlpp
@@ -17,8 +17,29 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test that DROP DDL statements are not executed in compile-only mode
+ */
 
-select value v from range(1,2) v where v > ?;
+-- param compile-only:string=true
+
+drop function test1.f1();
+
+drop synonym test1.s1;
+
+drop view test1.v1;
+
+drop index test1.ds1.idx1;
+
+drop dataset test1.ds1;
+
+drop type test1.t1;
+
+
+
+
+
+
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.4.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.4.query.sqlpp
index 6095b26..5760b8e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.4.query.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Check that objects were not dropped
+ */
 
-select value v from range(1,2) v where v > ?;
+select k, n
+from test1.f1() t
+where dv like "test1"
+order by k, n;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.5.ddl.sqlpp
similarity index 87%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.5.ddl.sqlpp
index 6095b26..37146d7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.5.ddl.sqlpp
@@ -17,8 +17,10 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test that DROP DDL statements are not executed in compile-only mode
+ */
 
-select value v from range(1,2) v where v > ?;
+-- param compile-only:string=true
+
+drop dataverse test1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.6.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.6.query.sqlpp
index 6095b26..5760b8e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.6.query.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Check that objects were not dropped
+ */
 
-select value v from range(1,2) v where v > ?;
+select k, n
+from test1.f1() t
+where dv like "test1"
+order by k, n;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.7.ddl.sqlpp
similarity index 70%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.7.ddl.sqlpp
index 6095b26..9b41631 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.7.ddl.sqlpp
@@ -17,8 +17,27 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test that CREATE DDL statements are not executed in compile-only mode
+ */
 
-select value v from range(1,2) v where v > ?;
+-- param compile-only:string=true
+
+use test1;
+
+create type t2 as closed {
+  c1 : bigint,
+  c2 : bigint
+};
+
+create dataset ds2(t2) primary key c1;
+
+create index idx2 on ds2(c2);
+
+create view v2 as select value v from ds2 v;
+
+create synonym s2 for v2;
+
+create function f2() {
+  select value v from v2 v
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.8.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.8.query.sqlpp
index 6095b26..c5afa49 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.8.query.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Check that objects were not created
+ */
 
-select value v from range(1,2) v where v > ?;
+select k, n
+from test1.f1() t
+where dv like "test1"
+order by k, n;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.9.ddl.sqlpp
similarity index 85%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.9.ddl.sqlpp
index 6095b26..7fb26db 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.9.ddl.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test that CREATE DDL statements are not executed in compile-only mode
+ */
 
-select value v from range(1,2) v where v > ?;
+-- param compile-only:string=true
+
+drop dataverse test2 if exists;
+create dataverse test2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.ddl.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.ddl.sqlpp
index 6095b26..82e9f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.ddl.sqlpp
@@ -17,8 +17,7 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+drop dataverse test1 if exists;
+create dataverse test1;
 
-select value v from range(1,2) v where v > ?;
+create dataset test1.t1(c1 int not unknown, c2 int) primary key c1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.2.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.2.plans.sqlpp
index 2a4952f..6095b26 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.2.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.2.plans.sqlpp
@@ -17,11 +17,8 @@
  * under the License.
  */
 
-/*
- * Test additional information returned when client-type=jdbc
- */
-
--- param client-type:string=jdbc
 -- param compile-only:string=true
+-- param logical-plan:string=true
+-- param plan-format:string=json
 
-select v from range(1,2) v where v between ? and ? ;
+select value v from range(1,2) v where v > ?;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.3.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.3.plans.sqlpp
index d7217a4..2a4952f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.3.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.3.plans.sqlpp
@@ -18,10 +18,10 @@
  */
 
 /*
- * Test additional information returned when client-type=jdbc (with explain)
+ * Test additional information returned when client-type=jdbc
  */
 
 -- param client-type:string=jdbc
 -- param compile-only:string=true
 
-explain select v from range(1,2) v where v between ? and ? ;
+select v from range(1,2) v where v between ? and ? ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.4.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.4.plans.sqlpp
index 8d2bd74..d7217a4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.4.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.4.plans.sqlpp
@@ -18,16 +18,10 @@
  */
 
 /*
- * Test additional information returned when client-type=jdbc (update statement)
+ * Test additional information returned when client-type=jdbc (with explain)
  */
 
 -- param client-type:string=jdbc
 -- param compile-only:string=true
 
-drop dataverse test1 if exists;
-
-create dataverse test1;
-
-create dataset test1.t1(c1 int not unknown, c2 int) primary key c1;
-
-insert into test1.t1 [{"c1": 1, "c2": ? }, {"c1": 3, "c2": ? }];
+explain select v from range(1,2) v where v between ? and ? ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.5.plans.sqlpp
similarity index 81%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.5.plans.sqlpp
index 6095b26..5947c5d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.5.plans.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test additional information returned when client-type=jdbc (update statement)
+ */
 
-select value v from range(1,2) v where v > ?;
+-- param client-type:string=jdbc
+-- param compile-only:string=true
+
+insert into test1.t1 [{"c1": 1, "c2": ? }, {"c1": 3, "c2": ? }];
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 47eb4395..9a5e17e 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
@@ -20,44 +20,52 @@
     <test-case FilePath="async-deferred">
         <compilation-unit name="async-failed">
             <output-dir compare="Clean-JSON">async-failed</output-dir>
+            <parameter name="profile" value="timings" type="string"/>
             <expected-error>Injected failure in inject-failure</expected-error>
         </compilation-unit>
     </test-case>
     <test-case FilePath="async-deferred">
         <compilation-unit name="async-compilation-failed">
+            <parameter name="profile" value="timings" type="string"/>
             <output-dir compare="Clean-JSON">async-compilation-failed</output-dir>
             <expected-error>Cannot find dataset gargel</expected-error>
         </compilation-unit>
     </test-case>
     <test-case FilePath="async-deferred">
         <compilation-unit name="deferred">
+            <parameter name="profile" value="timings" type="string"/>
             <output-dir compare="Clean-JSON">deferred</output-dir>
         </compilation-unit>
     </test-case>
     <test-case FilePath="async-deferred">
         <compilation-unit name="async">
+            <parameter name="profile" value="timings" type="string"/>
             <output-dir compare="Clean-JSON">async</output-dir>
         </compilation-unit>
     </test-case>
     <test-case FilePath="async-deferred">
         <compilation-unit name="async-repeated">
+            <parameter name="profile" value="timings" type="string"/>
             <output-dir compare="Clean-JSON">async-repeated</output-dir>
         </compilation-unit>
     </test-case>
     <test-case FilePath="async-deferred">
         <compilation-unit name="async-running">
+            <parameter name="profile" value="timings" type="string"/>
             <output-dir compare="Clean-JSON">async-running</output-dir>
         </compilation-unit>
     </test-case>
     <test-case FilePath="async-deferred">
         <compilation-unit name="async-exhausted-result">
             <output-dir compare="Clean-JSON">async-exhausted-result</output-dir>
+            <parameter name="profile" value="timings" type="string"/>
             <expected-error>Premature end of chunk</expected-error> <!--TODO:REVISIT -->
             <source-location>false</source-location>
         </compilation-unit>
     </test-case>
     <test-case FilePath="async-deferred">
         <compilation-unit name="async-json">
+            <parameter name="profile" value="timings" type="string"/>
             <output-dir compare="Clean-JSON">async-json</output-dir>
         </compilation-unit>
     </test-case>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/incomparable_types/incomparable_types.003.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/incomparable_types/incomparable_types.003.query.sqlpp
index 66e6595..c9f0619 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/incomparable_types/incomparable_types.003.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/incomparable_types/incomparable_types.003.query.sqlpp
@@ -17,10 +17,11 @@
  * under the License.
  */
 
+set `compiler.parallelism` "1";
 -- param max-warnings:string=10
 
 use test;
 
 from ds1 join ds2 on ds1.f /*+ hash-bcast */ = ds2.f
 where ds1.f > 1
-select ds1.f;
\ No newline at end of file
+select ds1.f;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.1.ddl.sqlpp
new file mode 100644
index 0000000..e1d6b10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.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.
+ */
+
+/*
+ * Description: Test ANALYZE DATASET statement
+ */
+
+set `import-private-functions` `true`;
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create function listMetadata(showSourceAvgItemSize, showSeed) {
+  select i.DatasetName, i.IndexName, i.SampleCardinalityTarget, i.SourceCardinality,
+    case when showSourceAvgItemSize then i.SourceAvgItemSize else i.SourceAvgItemSize > 0 end as SourceAvgItemSize,
+    case when showSeed then i.SampleSeed else i.SampleSeed is known end as SampleSeed
+  from Metadata.`Index` i
+  where i.DataverseName = "test" and i.IndexName like "sample_idx%"
+  order by i.IndexName
+};
+
+create function showSampleStats(dsName, idxName, showMinMax) {
+  select count(v.values) as cnt,
+    case when showMinMax then min(v.values[0]) else min(v.values[0]) > 0 end as min_pk,
+    case when showMinMax then max(v.values[0]) else max(v.values[0]) > 0 end as max_pk,
+    case when showMinMax then min(v.values[1].x) else min(v.values[1].x) < 0 end as min_x,
+    case when showMinMax then max(v.values[1].x) else max(v.values[1].x) < 0 end as max_x
+  from dump_index("test", dsName, idxName) as v
+};
+
+create type t1 as open {
+  id : bigint
+};
+
+create dataset ds1(t1) primary key id;
+
+-- analyze on an empty dataset
+
+analyze dataset ds1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.10.ddl.sqlpp
similarity index 78%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.10.ddl.sqlpp
index 6095b26..da5fe13 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.10.ddl.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: analyze dataset with sample=medium.
+ * Note, there are fewer tuples in the dataset that the target sample size
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+analyze dataset test.ds1 with { "sample": "medium", "sample-seed": 234.0 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.11.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.11.query.sqlpp
index 6095b26..38ded0a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.11.query.sqlpp
@@ -17,8 +17,14 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was re-created
+ */
 
-select value v from range(1,2) v where v > ?;
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+  listMetadata(false, true) metadata,
+  showSampleStats("ds1", "sample_idx_2_ds1", true) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.12.update.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.12.update.sqlpp
index 6095b26..0e3886d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.12.update.sqlpp
@@ -17,8 +17,12 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: Insert more data
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+insert into ds1
+select element {'id':x, 'x':-x }
+from range(1101, 4400) as x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.13.ddl.sqlpp
similarity index 80%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.13.ddl.sqlpp
index 6095b26..b098016 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.13.ddl.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: analyze dataset with sample=medium
+ * Note, there are more tuples in the dataset that the target sample size
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+analyze dataset ds1 with { "sample": "medium" };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.14.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.14.query.sqlpp
index 6095b26..4cae202 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.14.query.sqlpp
@@ -17,8 +17,14 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was re-created
+ */
 
-select value v from range(1,2) v where v > ?;
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+  listMetadata(false, false) metadata,
+  showSampleStats("ds1", "sample_idx_1_ds1", false) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.15.ddl.sqlpp
similarity index 78%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.15.ddl.sqlpp
index 6095b26..6ceb8141 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.15.ddl.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: analyze dataset with sample=high.
+ * Note, there are fewer tuples in the dataset that the target sample size
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+analyze dataset ds1 with { "sample": "high", "sample-seed": "345" };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.16.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.16.query.sqlpp
index 6095b26..38ded0a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.16.query.sqlpp
@@ -17,8 +17,14 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was re-created
+ */
 
-select value v from range(1,2) v where v > ?;
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+  listMetadata(false, true) metadata,
+  showSampleStats("ds1", "sample_idx_2_ds1", true) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.17.update.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.17.update.sqlpp
index 6095b26..0c5f8ad 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.17.update.sqlpp
@@ -17,8 +17,12 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: Insert more data
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+insert into ds1
+select element {'id':x, 'x':-x }
+from range(4401, 17100) as x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.18.ddl.sqlpp
similarity index 80%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.18.ddl.sqlpp
index 6095b26..4385f95 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.18.ddl.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: analyze dataset with sample=high
+ * Note, there are more tuples in the dataset that the target sample size
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+analyze dataset ds1 with { "sample": "high" };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.19.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.19.query.sqlpp
index 6095b26..4cae202 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.19.query.sqlpp
@@ -17,8 +17,14 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was re-created
+ */
 
-select value v from range(1,2) v where v > ?;
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+  listMetadata(false, false) metadata,
+  showSampleStats("ds1", "sample_idx_1_ds1", false) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.2.query.sqlpp
similarity index 83%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.2.query.sqlpp
index 6095b26..e786e0e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.2.query.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: Check that the sample index was created even thought
+ *              the source dataset is empty
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+listMetadata(true, false);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.20.ddl.sqlpp
similarity index 83%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.20.ddl.sqlpp
index 6095b26..3190b2d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.20.ddl.sqlpp
@@ -17,8 +17,10 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: test that the sample index is dropped using "analyze dataset drop statistics" statement
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+analyze dataset ds1 drop statistics;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.21.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.21.query.sqlpp
index 6095b26..759fc3f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.21.query.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was dropped
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+select count(*) cnt
+from listMetadata(true, false) v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.22.ddl.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.22.ddl.sqlpp
index 6095b26..7d6bf92 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.22.ddl.sqlpp
@@ -17,8 +17,6 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+use test;
 
-select value v from range(1,2) v where v > ?;
+analyze dataset ds1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.23.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.23.query.sqlpp
index 6095b26..a593df3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.23.query.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was re-created again
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+select count(*) cnt
+from listMetadata(true, false) v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.24.ddl.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.24.ddl.sqlpp
index 6095b26..151309a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.24.ddl.sqlpp
@@ -17,8 +17,10 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: test that the sample index is dropped when its source dataset is dropped
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+drop dataset ds1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.25.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.25.query.sqlpp
index 6095b26..759fc3f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.25.query.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was dropped
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+select count(*) cnt
+from listMetadata(true, false) v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.3.update.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.3.update.sqlpp
index 6095b26..1b83b99 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.3.update.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+use test;
 
-select value v from range(1,2) v where v > ?;
+insert into ds1
+select element {'id':x, 'x':-x }
+from range(1, 8) as x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.4.ddl.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.4.ddl.sqlpp
index 6095b26..ed97897 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.4.ddl.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: Test sample size parameter
+ */
 
-select value v from range(1,2) v where v > ?;
+analyze dataset test.ds1 with { "sample": "low", "sample-seed": 123 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.5.query.sqlpp
similarity index 72%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.5.query.sqlpp
index 6095b26..e0cd6cc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.5.query.sqlpp
@@ -17,8 +17,15 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that sample index was recreated with a new name.
+ * Also check that all 8 dataset tuples are in the sample
+ */
 
-select value v from range(1,2) v where v > ?;
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+  listMetadata(false, true) metadata,
+  showSampleStats("ds1", "sample_idx_2_ds1", true) stats
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.6.update.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.6.update.sqlpp
index 6095b26..340fbb3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.6.update.sqlpp
@@ -17,8 +17,12 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: Insert more data
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+insert into ds1
+select element {'id':x, 'x':-x }
+from range(9, 1100) as x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.7.query.sqlpp
similarity index 72%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.7.query.sqlpp
index 6095b26..c4930b0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.7.query.sqlpp
@@ -17,8 +17,15 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample content remains the same
+ * after INSERT because we did not run ANALYZE DATASET after that
+ */
 
-select value v from range(1,2) v where v > ?;
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+  listMetadata(false, true) metadata,
+  showSampleStats("ds1", "sample_idx_2_ds1", true) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.8.ddl.sqlpp
similarity index 80%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.8.ddl.sqlpp
index 6095b26..2f452f0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.8.ddl.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: analyze dataset with sample=low.
+ * Note, there are more tuples in the dataset that the target sample size
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+analyze dataset ds1 with { "sample": "low" };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.9.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.9.query.sqlpp
index 6095b26..4cae202 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.9.query.sqlpp
@@ -17,8 +17,14 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was re-created
+ */
 
-select value v from range(1,2) v where v > ?;
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+  listMetadata(false, false) metadata,
+  showSampleStats("ds1", "sample_idx_1_ds1", false) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.1.ddl.sqlpp
similarity index 70%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.1.ddl.sqlpp
index 6095b26..1bea9c2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.1.ddl.sqlpp
@@ -17,8 +17,14 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test plan when right branch of an inner join uses an outer variable.
+ * Currently this results in NL join
+ */
 
-select value v from range(1,2) v where v > ?;
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create dataset t1(id bigint not unknown) open type primary key id;
+create dataset t2(id uuid not unknown) open type primary key id autogenerated;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.2.update.sqlpp
similarity index 68%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.2.update.sqlpp
index 6095b26..7f3bae1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.2.update.sqlpp
@@ -17,8 +17,18 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test plan when right branch of an inner join uses an outer variable.
+ * Currently this results in NL join
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+insert into t1([
+  {"id": 1, "x": [{"b": 1}, {"b": 2}, {"b": 1}, {"b": 2}, {"b": 4}, {"b": 3}, {"b": 5}] },
+  {"id": 2, "x": [{"b": 9}, {"b": 2}, {"b": 7}, {"b": 2}, {"b": 4}, {"b": 3}, {"b": 5}] }
+]);
+insert into t2([
+  {"y": 1},
+  {"y": 3}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.3.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.3.query.sqlpp
index 6095b26..aea825f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/inner_right_corr/inner_right_corr.3.query.sqlpp
@@ -17,8 +17,14 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test plan when right branch of an inner join uses an outer variable.
+ * Currently this results in NL join
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+select a
+from t1
+let a = (select value count(*) from t2 join t1.x as z on t2.y = z.b )
+order by t1.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.1.ddl.sqlpp
new file mode 100644
index 0000000..ab0aa32
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.1.ddl.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.
+ */
+
+/*
+ * Test the fix for nested loop join delivered partitioning property ASTERIXDB-3066
+ */
+
+DROP DATAVERSE tpch IF EXISTS;
+CREATE DATAVERSE tpch;
+
+USE tpch;
+
+CREATE TYPE tpch.SupplierType AS
+ CLOSED {
+  s_suppkey : bigint,
+  s_name : string,
+  s_address : string,
+  s_nationkey : bigint,
+  s_phone : string,
+  s_acctbal : double,
+  s_comment : string
+};
+
+CREATE TYPE tpch.PartType AS
+ CLOSED {
+  p_partkey : bigint,
+  p_name : string,
+  p_mfgr : string,
+  p_brand : string,
+  p_type : string,
+  p_size : bigint,
+  p_container : string,
+  p_retailprice : double,
+  p_comment : string
+};
+
+CREATE TYPE tpch.PartSuppType AS
+ CLOSED {
+  ps_partkey : bigint,
+  ps_suppkey : bigint,
+  ps_availqty : bigint,
+  ps_supplycost : double,
+  ps_comment : string
+};
+
+CREATE DATASET Supplier(SupplierType) PRIMARY KEY s_suppkey;
+CREATE DATASET Part(PartType) PRIMARY KEY p_partkey;
+CREATE DATASET Partsupp(PartSuppType) PRIMARY KEY ps_partkey,ps_suppkey;
+
+CREATE INDEX partsupp_fk_part ON Partsupp (ps_partkey);
+CREATE INDEX partsupp_fk_supplier ON Partsupp (ps_suppkey);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.2.update.sqlpp
similarity index 65%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.2.update.sqlpp
index 6095b26..b7396c3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.2.update.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+USE tpch;
 
-select value v from range(1,2) v where v > ?;
+LOAD DATASET Supplier USING localfs ((`path`=`asterix_nc1://data/tpch0.001/supplier.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+LOAD DATASET Part USING localfs ((`path`=`asterix_nc1://data/tpch0.001/part.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+LOAD DATASET Partsupp USING localfs ((`path`=`asterix_nc1://data/tpch0.001/partsupp.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.3.query.sqlpp
similarity index 81%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.3.query.sqlpp
index 6095b26..ec57b20 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.3.query.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+USE tpch;
 
-select value v from range(1,2) v where v > ?;
+FROM Supplier s JOIN (FROM Partsupp ps, Part p SELECT ps.ps_suppkey AS suppkey) AS p_ps
+ON s.s_suppkey /*+ hash-bcast */ = p_ps.suppkey
+SELECT count(*) AS count;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.4.query.sqlpp
similarity index 83%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.4.query.sqlpp
index 6095b26..20eeff2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.4.query.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+USE tpch;
 
-select value v from range(1,2) v where v > ?;
+FROM Supplier s JOIN (FROM Partsupp ps, Part p SELECT ps.ps_suppkey AS suppkey) AS p_ps
+ON s.s_suppkey = p_ps.suppkey
+SELECT count(*) AS count;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.5.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.5.query.sqlpp
index 6095b26..cb3e800 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj_partitioning_property/nlj_partitioning_property.5.query.sqlpp
@@ -17,8 +17,7 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+USE tpch;
 
-select value v from range(1,2) v where v > ?;
+FROM Supplier s JOIN (FROM Partsupp ps, Part p SELECT ps.ps_suppkey AS suppkey) AS p_ps ON TRUE
+SELECT count(*) AS count;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/metadata_only_02/metadata_only_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/metadata_only_02/metadata_only_02.3.query.sqlpp
new file mode 100644
index 0000000..11c7325
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/metadata_only_02/metadata_only_02.3.query.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.
+ */
+
+/*
+ * test fix for ASTERIXDB-3075
+ */
+
+LET dv_name = ["test2", "test1"], ds_name = ["ds2", "ds1"], syn_name = [ "syn2", "syn1" ],
+synonym_names = (SELECT s.SynonymName, s.ObjectName
+                 FROM Metadata.`Synonym` s, syn_name
+                 WHERE s.SynonymName = syn_name),
+
+dataset_ds_dv_names = (SELECT d.DatasetName, d.DataverseName
+                       FROM Metadata.`Dataset` d, (SELECT * FROM ds_name, dv_name) AS ds_dv_names
+                       WHERE d.DatasetName = ds_dv_names.ds_name AND d.DataverseName = ds_dv_names.dv_name),
+
+dataset_dv_ds_names = (SELECT d.DatasetName, d.DataverseName
+                       FROM Metadata.`Dataset` d, (SELECT * FROM dv_name, ds_name) AS dv_ds_names
+                       WHERE d.DatasetName = dv_ds_names.ds_name AND d.DataverseName = dv_ds_names.dv_name),
+
+left_branch = (SELECT s.SynonymName, s.DataverseName, s.ObjectName
+               FROM Metadata.`Synonym` s LEFT OUTER JOIN dataset_dv_ds_names
+               ON dataset_dv_ds_names.DatasetName = s.ObjectName
+               ORDER BY s.DataverseName, s.SynonymName),
+
+right_branch = (SELECT synonym_names.SynonymName
+                FROM synonym_names LEFT OUTER JOIN dataset_ds_dv_names
+                ON dataset_ds_dv_names.DatasetName = synonym_names.ObjectName)
+
+FROM left_branch lb LEFT OUTER JOIN right_branch rb
+ON lb.ObjectName = rb.SynonymName
+SELECT lb.DataverseName, lb.SynonymName
+ORDER BY lb.DataverseName, lb.SynonymName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/metadata_only_02/metadata_only_02.9.ddl.sqlpp
similarity index 83%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/metadata_only_02/metadata_only_02.9.ddl.sqlpp
index 6095b26..bd99c9d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/metadata_only_02/metadata_only_02.9.ddl.sqlpp
@@ -17,8 +17,5 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
+drop dataverse test1 if exists;
+drop dataverse test2 if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue453-2/query-issue453-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue453-2/query-issue453-2.3.query.sqlpp
index 31e68b0..0a81db6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue453-2/query-issue453-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue453-2/query-issue453-2.3.query.sqlpp
@@ -25,8 +25,7 @@
 
 use test;
 
-
-select element d
+select d.int_m, d.int_o, d.string_m, d.string_o
 from  DataOpen as d
-order by d.id
+order by d.int_m, d.int_o, d.string_m, d.string_o;
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.1.ddl.sqlpp
similarity index 72%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.1.ddl.sqlpp
index 6095b26..8fc7172 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.1.ddl.sqlpp
@@ -17,8 +17,13 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+DROP DATAVERSE ComplexExclude IF EXISTS;
+CREATE DATAVERSE ComplexExclude;
+USE ComplexExclude;
 
-select value v from range(1,2) v where v > ?;
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+
+CREATE VIEW UsersWithoutPII AS
+    FROM ComplexExclude.Users U
+    SELECT U.* EXCLUDE phones, address, ssn;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.10.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.10.query.sqlpp
index 6095b26..67d91fb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.10.query.sqlpp
@@ -17,8 +17,9 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
+//         Query with RIGHT JOIN and EXCLUDE.
+FROM       ComplexExclude.Users U2
+RIGHT JOIN ComplexExclude.Users U1
+ON         U1.best_friend = U2.user_id
+SELECT     * EXCLUDE U1.address, U2.address
+ORDER BY   U1.user_id, U2.user_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.11.query.sqlpp
similarity index 80%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.11.query.sqlpp
index 6095b26..099c6c5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.11.query.sqlpp
@@ -17,8 +17,7 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
+//     Query with DISTINCT (applies after EXCLUDE) and normal WHERE.
+FROM   ComplexExclude.Users U
+WHERE  U.best_friend = 1
+SELECT DISTINCT U.* EXCLUDE address, favorite_color, user_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.12.query.sqlpp
similarity index 83%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.12.query.sqlpp
index 6095b26..6b5ac14 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.12.query.sqlpp
@@ -17,8 +17,7 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
+//       Query using view with EXCLUDE + EXCLUDE on top.
+FROM     ComplexExclude.UsersWithoutPII U
+SELECT   U.* EXCLUDE favorite_color
+ORDER BY U.user_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.13.query.sqlpp
similarity index 71%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.13.query.sqlpp
index 6095b26..46bac5d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.13.query.sqlpp
@@ -17,8 +17,15 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
+//       Query using EXCLUDE as a field-name.
+//       (We disallow EXCLUDE as a field-name if not preceded with an 'AS').
+WITH     A AS [ { "a": 1, "b": 2, "EXCLUDE": 3 } ]
+FROM     A
+SELECT   A AS EXCLUDE
+UNION ALL
+FROM     A
+SELECT   A.EXCLUDE
+UNION ALL
+FROM     A
+SELECT   A EXCLUDE A.a
+ORDER BY EXCLUDE;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.14.query.sqlpp
new file mode 100644
index 0000000..20975a4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.14.query.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.
+ */
+
+//                Query using all clauses + EXCLUDE.
+WITH              otherUsers AS ( FROM ComplexExclude.Users U SELECT VALUE U )
+FROM              ComplexExclude.Users U1
+JOIN              otherUsers U2
+ON                U2.user_id = U1.best_friend
+LEFT UNNEST       U1.phones U1P
+LET               bestFriend = U2
+WHERE             U1.user_id = U2.best_friend
+GROUP BY          U1
+GROUP AS          G
+LET               bestFriends = ( FROM G SELECT VALUE bestFriend )
+HAVING            COUNT(*) > 0
+SELECT            bestFriends, U1.* EXCLUDE address, phones
+ORDER BY          U1.user_id
+LIMIT             10;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.2.update.sqlpp
similarity index 65%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.2.update.sqlpp
index 6095b26..4c770f6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.2.update.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+USE ComplexExclude;
 
-select value v from range(1,2) v where v > ?;
+INSERT INTO Users [
+  { "user_id": 1, "best_friend": 2, "phones": [ { "kind": "MOBILE", "number": "222-222-2222" } ] },
+  { "user_id": 2, "best_friend": 1, "address": { "zip_code": "99929", "street": "2341 Apple Street" } },
+  { "user_id": 3, "best_friend": 1, "address": { "zip_code": "99929", "street": "2341 Apple Street" }, "favorite_color": "Green" },
+  { "user_id": 4, "best_friend": null }
+];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.3.query.sqlpp
similarity index 83%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.3.query.sqlpp
index 6095b26..bc01577 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.3.query.sqlpp
@@ -17,8 +17,7 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
+//       Query using view with EXCLUDE.
+FROM     ComplexExclude.UsersWithoutPII U
+SELECT   U.*
+ORDER BY U.user_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.4.query.sqlpp
similarity index 81%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.4.query.sqlpp
index 6095b26..6aa8a33 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.4.query.sqlpp
@@ -17,8 +17,7 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
+//       Query with extraneous EXCLUDE terms ("address.zip_code" is unnecessary).
+FROM     ComplexExclude.Users U
+SELECT   U.* EXCLUDE address, address.zip_code
+ORDER BY U.user_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.5.query.sqlpp
similarity index 74%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.5.query.sqlpp
index 6095b26..8ef1df6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.5.query.sqlpp
@@ -17,8 +17,10 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
+//       Query with EXCLUDE in subquery.
+FROM     ComplexExclude.Users U1
+WHERE    U1.best_friend = 1
+SELECT   VALUE ( FROM   ComplexExclude.Users U2
+                 WHERE  U2.user_id = U1.user_id
+                 SELECT U2.* EXCLUDE address )
+ORDER BY U1.user_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.6.query.sqlpp
similarity index 81%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.6.query.sqlpp
index 6095b26..8e0b660 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.6.query.sqlpp
@@ -17,8 +17,7 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
+//       Query with EXCLUDE on the projections w/o star.
+FROM     ComplexExclude.Users U
+SELECT   U.user_id, U.best_friend, U.address EXCLUDE best_friend, address
+ORDER BY U.user_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.7.query.sqlpp
similarity index 82%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.7.query.sqlpp
index 6095b26..ba6588e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.7.query.sqlpp
@@ -17,8 +17,9 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
+//       Query with GROUP-BY and EXCLUDE.
+FROM     ComplexExclude.Users U
+GROUP BY U.best_friend
+GROUP AS G
+SELECT   * EXCLUDE G
+ORDER BY U.best_friend NULLS LAST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.8.query.sqlpp
similarity index 70%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.8.query.sqlpp
index 6095b26..437c7af 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.8.query.sqlpp
@@ -17,8 +17,13 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
+//         Query with CTE, EXCLUDE, and explicit JOIN.
+LET        SpecialUsers = (
+               FROM   ComplexExclude.Users U
+               WHERE  U.user_id = 1
+               SELECT VALUE U
+           )
+FROM       ComplexExclude.Users U
+INNER JOIN SpecialUsers S
+ON         U.user_id = S.user_id
+SELECT     U.* EXCLUDE phones;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.9.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.9.query.sqlpp
index 6095b26..ff507a3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.9.query.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
+//       Query with LET after the FROM clause.
+FROM     ComplexExclude.Users U
+LET      miscInfo = { "24as": "23412", "address": "2341 Orange Street" }
+SELECT   * EXCLUDE U.address, miscInfo.address, U.phones
+ORDER BY U.user_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.1.ddl.sqlpp
similarity index 79%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.1.ddl.sqlpp
index 6095b26..7c54e82 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.1.ddl.sqlpp
@@ -17,8 +17,9 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+DROP DATAVERSE ExcludeNegative IF EXISTS;
+CREATE DATAVERSE ExcludeNegative;
+USE ExcludeNegative;
 
-select value v from range(1,2) v where v > ?;
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.2.query.sqlpp
similarity index 83%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.2.query.sqlpp
index 6095b26..8e8c021 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.2.query.sqlpp
@@ -17,8 +17,6 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
+//       EXCLUDE cannot be used as an alias without 'AS'.
+FROM     NegativeExclude.Users U
+SELECT   U.user_id EXCLUDE;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.3.query.sqlpp
similarity index 83%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.3.query.sqlpp
index 6095b26..eee1023 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.3.query.sqlpp
@@ -17,8 +17,6 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
+//       EXCLUDE cannot be used with SELECT VALUE.
+FROM     NegativeExclude.Users U
+SELECT   VALUE U EXCLUDE user_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.1.ddl.sqlpp
new file mode 100644
index 0000000..3cb07d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE TinySocial IF EXISTS;
+CREATE DATAVERSE TinySocial;
+USE TinySocial;
+
+CREATE TYPE TwitterUserType AS {
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint
+};
+
+CREATE TYPE TweetMessageType AS {
+  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/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.2.update.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.2.update.sqlpp
index 6095b26..a60ef4d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.2.update.sqlpp
@@ -17,8 +17,6 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
+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/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.3.query.sqlpp
similarity index 65%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.3.query.sqlpp
index 6095b26..4d7489a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.3.query.sqlpp
@@ -17,8 +17,20 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+FROM     TinySocial.TweetMessages TM
 
-select value v from range(1,2) v where v > ?;
+         // SELECT-var.*.
+SELECT   TM.*
+
+         // Nested field that does not exist.
+EXCLUDE   user.does_not_exist,
+         // Nested field that does exist.
+         user.lang,
+         // Field with quotes.
+         `send-time`,
+         // Non-nested field that does not exist.
+         does_not_exist,
+         // Non-nested field that exists.
+         tweetid
+
+ORDER BY TM.tweetid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.4.query.sqlpp
similarity index 65%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.4.query.sqlpp
index 6095b26..06c6df0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.4.query.sqlpp
@@ -17,8 +17,19 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+FROM     TinySocial.TweetMessages TM,
+         TinySocial.TwitterUsers TU
 
-select value v from range(1,2) v where v > ?;
+WHERE    TM.user.`screen-name` = TU.`screen-name`
+
+         // SELECT * returns { "TM": ..., "TU": ... }
+SELECT   *
+
+         // Identifier that exists in scope.
+EXCLUDE   TU,
+         // Nested identifier that exists in scope.
+         TM.user.`screen-name`,
+         // Identifier that does not exist in scope.
+         TI
+
+ORDER BY TM.tweetid;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.5.query.sqlpp
similarity index 73%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.5.query.sqlpp
index 6095b26..d41b3a7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.5.query.sqlpp
@@ -17,8 +17,17 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+FROM     TinySocial.TweetMessages TM1
+SELECT   TM1.*
+EXCLUDE   user.does_not_exist,
+         user.lang,
+         `send-time`,
+         does_not_exist
 
-select value v from range(1,2) v where v > ?;
+UNION ALL
+
+FROM     TinySocial.TweetMessages TM2
+SELECT   TM2.*
+EXCLUDE   user.lang
+
+ORDER BY tweetid, `send-time` NULLS FIRST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.6.query.sqlpp
similarity index 68%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.6.query.sqlpp
index 6095b26..e5453ef 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.6.query.sqlpp
@@ -17,8 +17,16 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+FROM     TinySocial.TweetMessages TM
 
-select value v from range(1,2) v where v > ?;
+         // SELECT-* (single-variable rule case).
+SELECT   *
+
+         // Nested field that exists, anchored by TM.
+EXCLUDE  TM.user.lang,
+         // Field with quotes (single-variable rule applies).
+         `send-time`,
+         // Non-nested field that exists (single-variable rule applies).
+         tweetid
+
+ORDER BY tweetid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.4.query.sqlpp
similarity index 72%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.4.query.sqlpp
index 6095b26..1f7c7d5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.4.query.sqlpp
@@ -17,8 +17,14 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+// see ASTERIXDB-3043
+SET `compiler.joinmemory` "160KB";
 
-select value v from range(1,2) v where v > ?;
+USE tpcds;
+
+SELECT count (*) AS cnt
+FROM customer c, customer_demographics cd2, customer_address ca
+WHERE
+ c.c_current_cdemo_sk /*+ indexnl */ = cd2.cd_demo_sk
+ AND c.c_current_addr_sk /*+ indexnl */ = ca.ca_address_sk
+ AND c.c_birth_month in [4,5];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.5.query.sqlpp
similarity index 74%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.5.query.sqlpp
index 6095b26..3a7ae0b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.5.query.sqlpp
@@ -17,8 +17,14 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+// see ASTERIXDB-3043
+SET `compiler.joinmemory` "160KB";
 
-select value v from range(1,2) v where v > ?;
+USE tpcds;
+
+SELECT count (*) AS cnt
+FROM customer c, customer_demographics cd2, customer_address ca
+WHERE
+ c.c_current_cdemo_sk = cd2.cd_demo_sk
+ AND c.c_current_addr_sk = ca.ca_address_sk
+ AND c.c_birth_month in [4,5];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/count_dataset/count_dataset.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/count_dataset/count_dataset.1.adm
index 9a0762a..c2daf56 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/count_dataset/count_dataset.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/count_dataset/count_dataset.1.adm
@@ -1,24 +1,24 @@
-distribute result [$$25]
+distribute result [$$26]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    aggregate [$$25] <- [agg-sql-sum($$28)]
+    aggregate [$$26] <- [agg-sql-sum($$29)]
     -- AGGREGATE  |UNPARTITIONED|
-      aggregate [$$28] <- [agg-sql-count(1)]
+      aggregate [$$29] <- [agg-sql-count(1)]
       -- AGGREGATE  |PARTITIONED|
         exchange
-        -- SORT_MERGE_EXCHANGE [$$26(ASC) ]  |PARTITIONED|
-          project ([$$26])
+        -- SORT_MERGE_EXCHANGE [$$27(ASC) ]  |PARTITIONED|
+          project ([$$27])
           -- STREAM_PROJECT  |PARTITIONED|
-            select (and(ge($$24, 1), le($$24, 10)))
+            select (and(ge($$25, 1), le($$25, 10)))
             -- STREAM_SELECT  |PARTITIONED|
-              project ([$$26, $$24])
+              project ([$$27, $$25])
               -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$24] <- [$$Tweet.getField(1)]
+                assign [$$25] <- [$$Tweet.getField(1)]
                 -- ASSIGN  |PARTITIONED|
                   exchange
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$26, $$Tweet] <- Twitter.Tweet
+                    data-scan []<-[$$27, $$Tweet] <- Twitter.Tweet
                     -- DATASOURCE_SCAN  |PARTITIONED|
                       exchange
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/count_dataset/count_dataset.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/count_dataset/count_dataset.1.adm
index f56f3a1..ed78e19 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/count_dataset/count_dataset.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/count_dataset/count_dataset.1.adm
@@ -1,24 +1,24 @@
-distribute result [$$25]
+distribute result [$$26]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    aggregate [$$25] <- [agg-sum($$28)]
+    aggregate [$$26] <- [agg-sum($$29)]
     -- AGGREGATE  |UNPARTITIONED|
-      aggregate [$$28] <- [agg-count(1)]
+      aggregate [$$29] <- [agg-count(1)]
       -- AGGREGATE  |PARTITIONED|
         exchange
-        -- SORT_MERGE_EXCHANGE [$$26(ASC) ]  |PARTITIONED|
-          project ([$$26])
+        -- SORT_MERGE_EXCHANGE [$$27(ASC) ]  |PARTITIONED|
+          project ([$$27])
           -- STREAM_PROJECT  |PARTITIONED|
-            select (and(ge($$24, 1), le($$24, 10)))
+            select (and(ge($$25, 1), le($$25, 10)))
             -- STREAM_SELECT  |PARTITIONED|
-              project ([$$26, $$24])
+              project ([$$27, $$25])
               -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$24] <- [$$Tweet.getField(1)]
+                assign [$$25] <- [$$Tweet.getField(1)]
                 -- ASSIGN  |PARTITIONED|
                   exchange
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$26, $$Tweet] <- Twitter.Tweet
+                    data-scan []<-[$$27, $$Tweet] <- Twitter.Tweet
                     -- DATASOURCE_SCAN  |PARTITIONED|
                       exchange
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
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 c55c0bc..d90e1f8 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
@@ -10,7 +10,11 @@
     "active\.suspend\.timeout" : 3600,
     "azure.request.timeout" : 120,
     "compiler\.arrayindex" : true,
+    "compiler.batch.lookup" : false,
+    "compiler.cbo" : false,
+    "compiler.cbotest" : true,
     "compiler\.external\.field\.pushdown" : true,
+    "compiler.forcejoinorder" : false,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
     "compiler\.indexonly" : true,
@@ -18,6 +22,7 @@
     "compiler\.joinmemory" : 262144,
     "compiler\.min\.memory\.allocation" : true,
     "compiler\.parallelism" : 0,
+    "compiler.queryplanshape" : "zigzag",
     "compiler\.sort\.parallel" : false,
     "compiler\.sort\.samples" : 100,
     "compiler\.sortmemory" : 327680,
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 661daf3..efa091c 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
@@ -10,7 +10,11 @@
     "active\.suspend\.timeout" : 3600,
     "azure.request.timeout" : 120,
     "compiler\.arrayindex" : true,
+    "compiler.batch.lookup" : false,
+    "compiler.cbo" : false,
+    "compiler.cbotest" : true,
     "compiler\.external\.field\.pushdown" : true,
+    "compiler.forcejoinorder" : false,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
     "compiler\.indexonly" : true,
@@ -18,6 +22,7 @@
     "compiler\.joinmemory" : 262144,
     "compiler\.min\.memory\.allocation" : true,
     "compiler\.parallelism" : -1,
+    "compiler.queryplanshape" : "zigzag",
     "compiler\.sort\.parallel" : true,
     "compiler\.sort\.samples" : 100,
     "compiler\.sortmemory" : 327680,
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 1f0e865..732f05c 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
@@ -10,7 +10,11 @@
     "active\.suspend\.timeout" : 3600,
     "azure.request.timeout" : 120,
     "compiler\.arrayindex" : true,
+    "compiler.batch.lookup" : false,
+    "compiler.cbo" : false,
+    "compiler.cbotest" : true,
     "compiler\.external\.field\.pushdown" : true,
+    "compiler.forcejoinorder" : false,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
     "compiler\.indexonly" : true,
@@ -18,6 +22,7 @@
     "compiler\.joinmemory" : 262144,
     "compiler\.min\.memory\.allocation" : true,
     "compiler\.parallelism" : 3,
+    "compiler.queryplanshape" : "zigzag",
     "compiler\.sort\.parallel" : true,
     "compiler\.sort\.samples" : 100,
     "compiler\.sortmemory" : 327680,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.10.adm
new file mode 100644
index 0000000..bacb60c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.10.adm
@@ -0,0 +1 @@
+{ "cnt": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.2.adm
new file mode 100644
index 0000000..c1ff5e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.2.adm
@@ -0,0 +1,6 @@
+{ "k": "Dataset", "n": "ds1" }
+{ "k": "Dataset", "n": "v1" }
+{ "k": "Datatype", "n": "t1" }
+{ "k": "Dataverse", "n": "test1" }
+{ "k": "Function", "n": "f1" }
+{ "k": "Synonym", "n": "s1" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.4.adm
new file mode 100644
index 0000000..c1ff5e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.4.adm
@@ -0,0 +1,6 @@
+{ "k": "Dataset", "n": "ds1" }
+{ "k": "Dataset", "n": "v1" }
+{ "k": "Datatype", "n": "t1" }
+{ "k": "Dataverse", "n": "test1" }
+{ "k": "Function", "n": "f1" }
+{ "k": "Synonym", "n": "s1" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.6.adm
new file mode 100644
index 0000000..c1ff5e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.6.adm
@@ -0,0 +1,6 @@
+{ "k": "Dataset", "n": "ds1" }
+{ "k": "Dataset", "n": "v1" }
+{ "k": "Datatype", "n": "t1" }
+{ "k": "Dataverse", "n": "test1" }
+{ "k": "Function", "n": "f1" }
+{ "k": "Synonym", "n": "s1" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.8.adm
new file mode 100644
index 0000000..c1ff5e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.8.adm
@@ -0,0 +1,6 @@
+{ "k": "Dataset", "n": "ds1" }
+{ "k": "Dataset", "n": "v1" }
+{ "k": "Datatype", "n": "t1" }
+{ "k": "Dataverse", "n": "test1" }
+{ "k": "Function", "n": "f1" }
+{ "k": "Synonym", "n": "s1" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.adm
deleted file mode 100644
index 1ced420..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.adm
+++ /dev/null
@@ -1 +0,0 @@
-{"statementCategory":"query","statementParameters":[1,2]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.1.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.regexjson
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.1.regexjson
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.regexjson
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.3.adm
index 83b47f4..1ced420 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.3.adm
@@ -1 +1 @@
-{"statementCategory":"query","statementParameters":[1,2],"explainOnly":true}
\ No newline at end of file
+{"statementCategory":"query","statementParameters":[1,2]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.4.adm
index 63c482a..83b47f4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.4.adm
@@ -1 +1 @@
-{"statementCategory":"update","statementParameters":[1,2]}
\ No newline at end of file
+{"statementCategory":"query","statementParameters":[1,2],"explainOnly":true}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.5.adm
new file mode 100644
index 0000000..63c482a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.5.adm
@@ -0,0 +1 @@
+{"statementCategory":"update","statementParameters":[1,2]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/request-param-validation-400-BAD/request-param-validation-400-BAD.01.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/request-param-validation-400-BAD/request-param-validation-400-BAD.01.regexjson
index ad95b7b..6fa99e2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/request-param-validation-400-BAD/request-param-validation-400-BAD.01.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/request-param-validation-400-BAD/request-param-validation-400-BAD.01.regexjson
@@ -6,6 +6,7 @@
 	"metrics": {
 		"elapsedTime": "R{.*}",
 		"executionTime": "R{.*}",
+		"compileTime": "R{.*}",
 		"resultCount": 0,
 		"resultSize": 0,
 		"processedObjects": 0,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-exhausted-result/async-exhausted-result.3.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-exhausted-result/async-exhausted-result.3.regexjson
index df5177b..7fe2d93 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-exhausted-result/async-exhausted-result.3.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-exhausted-result/async-exhausted-result.3.regexjson
@@ -11,5 +11,6 @@
 	  { "i": 9, "i2": 81 },
 	  { "i": 10, "i2": 100 }
   ],
-  "metrics": "R{.*}"
+  "metrics": "R{.*}",
+  "profile": "R{.*}"
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-repeated/async-repeated.3.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-repeated/async-repeated.3.regexjson
index df5177b..7fe2d93 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-repeated/async-repeated.3.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-repeated/async-repeated.3.regexjson
@@ -11,5 +11,6 @@
 	  { "i": 9, "i2": 81 },
 	  { "i": 10, "i2": 100 }
   ],
-  "metrics": "R{.*}"
+  "metrics": "R{.*}",
+  "profile": "R{.*}"
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-repeated/async-repeated.4.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-repeated/async-repeated.4.regexjson
index df5177b..7fe2d93 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-repeated/async-repeated.4.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-repeated/async-repeated.4.regexjson
@@ -11,5 +11,6 @@
 	  { "i": 9, "i2": 81 },
 	  { "i": 10, "i2": 100 }
   ],
-  "metrics": "R{.*}"
+  "metrics": "R{.*}",
+  "profile": "R{.*}"
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-running/async-running.4.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-running/async-running.4.regexjson
index a2a5f0a8..3e0193d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-running/async-running.4.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async-running/async-running.4.regexjson
@@ -2,5 +2,6 @@
 	"results": [
 	  "result"
   ],
-  "metrics": "R{.*}"
+  "metrics": "R{.*}",
+  "profile": "R{.*}"
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async/async.3.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async/async.3.regexjson
index df5177b..7fe2d93 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async/async.3.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/async/async.3.regexjson
@@ -11,5 +11,6 @@
 	  { "i": 9, "i2": 81 },
 	  { "i": 10, "i2": 100 }
   ],
-  "metrics": "R{.*}"
+  "metrics": "R{.*}",
+  "profile": "R{.*}"
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/deferred/deferred.2.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/deferred/deferred.2.regexjson
index df5177b..7fe2d93 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/deferred/deferred.2.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/async-deferred/deferred/deferred.2.regexjson
@@ -11,5 +11,6 @@
 	  { "i": 9, "i2": 81 },
 	  { "i": 10, "i2": 100 }
   ],
-  "metrics": "R{.*}"
+  "metrics": "R{.*}",
+  "profile": "R{.*}"
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.11.adm
new file mode 100644
index 0000000..58f454b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.11.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_2_ds1", "SampleCardinalityTarget": 4252, "SourceCardinality": 1100, "SourceAvgItemSize": true, "SampleSeed": 234 }, "stats": { "cnt": 1100, "min_pk": 1, "max_pk": 1100, "min_x": -1100, "max_x": -1 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.14.adm
new file mode 100644
index 0000000..6ef756a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.14.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_1_ds1", "SampleCardinalityTarget": 4252, "SourceCardinality": 4400, "SourceAvgItemSize": true, "SampleSeed": true }, "stats": { "cnt": 4246, "min_pk": true, "max_pk": true, "min_x": true, "max_x": true } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.16.adm
new file mode 100644
index 0000000..01eb5b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.16.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_2_ds1", "SampleCardinalityTarget": 17008, "SourceCardinality": 4400, "SourceAvgItemSize": true, "SampleSeed": 345 }, "stats": { "cnt": 4400, "min_pk": 1, "max_pk": 4400, "min_x": -4400, "max_x": -1 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.19.adm
new file mode 100644
index 0000000..60b969f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.19.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_1_ds1", "SampleCardinalityTarget": 17008, "SourceCardinality": 17100, "SourceAvgItemSize": true, "SampleSeed": true }, "stats": { "cnt": 16972, "min_pk": true, "max_pk": true, "min_x": true, "max_x": true } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.2.adm
new file mode 100644
index 0000000..e3cefee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.2.adm
@@ -0,0 +1 @@
+{ "DatasetName": "ds1", "IndexName": "sample_idx_1_ds1", "SampleCardinalityTarget": 1063, "SourceCardinality": 0, "SourceAvgItemSize": 0, "SampleSeed": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.21.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.21.adm
new file mode 100644
index 0000000..bacb60c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.21.adm
@@ -0,0 +1 @@
+{ "cnt": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.23.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.23.adm
new file mode 100644
index 0000000..f86e66b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.23.adm
@@ -0,0 +1 @@
+{ "cnt": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.25.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.25.adm
new file mode 100644
index 0000000..bacb60c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.25.adm
@@ -0,0 +1 @@
+{ "cnt": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.5.adm
new file mode 100644
index 0000000..605bb12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.5.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_2_ds1", "SampleCardinalityTarget": 1063, "SourceCardinality": 8, "SourceAvgItemSize": true, "SampleSeed": 123 }, "stats": { "cnt": 8, "min_pk": 1, "max_pk": 8, "min_x": -8, "max_x": -1 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.7.adm
new file mode 100644
index 0000000..605bb12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.7.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_2_ds1", "SampleCardinalityTarget": 1063, "SourceCardinality": 8, "SourceAvgItemSize": true, "SampleSeed": 123 }, "stats": { "cnt": 8, "min_pk": 1, "max_pk": 8, "min_x": -8, "max_x": -1 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.9.adm
new file mode 100644
index 0000000..0084d2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.9.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_1_ds1", "SampleCardinalityTarget": 1063, "SourceCardinality": 1100, "SourceAvgItemSize": true, "SampleSeed": true }, "stats": { "cnt": 1033, "min_pk": true, "max_pk": true, "min_x": true, "max_x": true } }
\ No newline at end of file
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 31ef021..beaa4f0 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,40 +1,40 @@
-distribute result [$$47]
+distribute result [$$50]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$47])
+    project ([$$50])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$47] <- [{"deptId": $#1, "star_cost": $$50}]
+      assign [$$50] <- [{"deptId": $#1, "star_cost": $$53}]
       -- ASSIGN  |PARTITIONED|
         exchange
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$#1 := $$55]) decor ([]) {
-                    aggregate [$$50] <- [agg-global-sql-sum($$54)]
+          group by ([$#1 := $$58]) decor ([]) {
+                    aggregate [$$53] <- [agg-global-sql-sum($$57)]
                     -- AGGREGATE  |LOCAL|
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                  }
-          -- SORT_GROUP_BY[$$55]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$58]  |PARTITIONED|
             exchange
-            -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
-              group by ([$$55 := $$48]) decor ([]) {
-                        aggregate [$$54] <- [agg-local-sql-sum($$45)]
+            -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+              group by ([$$58 := $$51]) decor ([]) {
+                        aggregate [$$57] <- [agg-local-sql-sum($$48)]
                         -- AGGREGATE  |LOCAL|
                           nested tuple source
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                      }
-              -- SORT_GROUP_BY[$$48]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$51]  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$45, $$48])
+                  project ([$$48, $$51])
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$48, $$45] <- [substring($$e.getField("dept").getField("department_id"), 0), $$e.getField("salary")]
+                    assign [$$51, $$48] <- [substring($$e.getField("dept").getField("department_id"), 0), $$e.getField("salary")]
                     -- ASSIGN  |PARTITIONED|
                       project ([$$e])
                       -- STREAM_PROJECT  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$49, $$e] <- gby.Employee
+                          data-scan []<-[$$52, $$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 bbdbb57..5069579 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 [$$46]
+distribute result [$$49]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$46])
+    project ([$$49])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$46] <- [{"deptId": $#1, "star_cost": $$49}]
+      assign [$$49] <- [{"deptId": $#1, "star_cost": $$52}]
       -- ASSIGN  |PARTITIONED|
         exchange
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$#1 := $$53]) decor ([]) {
-                    aggregate [$$49] <- [agg-global-sql-sum($$52)]
+          group by ([$#1 := $$56]) decor ([]) {
+                    aggregate [$$52] <- [agg-global-sql-sum($$55)]
                     -- AGGREGATE  |LOCAL|
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                  }
-          -- SORT_GROUP_BY[$$53]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$56]  |PARTITIONED|
             exchange
-            -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
-              group by ([$$53 := $$47]) decor ([]) {
-                        aggregate [$$52] <- [agg-local-sql-sum($$44)]
+            -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
+              group by ([$$56 := $$50]) decor ([]) {
+                        aggregate [$$55] <- [agg-local-sql-sum($$47)]
                         -- AGGREGATE  |LOCAL|
                           nested tuple source
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                      }
-              -- SORT_GROUP_BY[$$47]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$50]  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$44, $$47])
+                  project ([$$47, $$50])
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$47, $$44] <- [substring($$e.getField(1), 0), $$e.getField(2)]
+                    assign [$$50, $$47] <- [substring($$e.getField(1), 0), $$e.getField(2)]
                     -- ASSIGN  |PARTITIONED|
                       project ([$$e])
                       -- STREAM_PROJECT  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$48, $$e] <- gby.Employee
+                          data-scan []<-[$$51, $$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_object_constructor-01/explain_object_constructor-01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-01/explain_object_constructor-01.1.adm
index 9a536a6..e89d96a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-01/explain_object_constructor-01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-01/explain_object_constructor-01.1.adm
@@ -2,7 +2,7 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    assign [$$2] <- [{ a: "1", b: 1 }]
+    assign [$$2] <- [{ "a": "1", "b": 1 }]
     -- ASSIGN  |UNPARTITIONED|
       empty-tuple-source
       -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-02/explain_object_constructor-02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-02/explain_object_constructor-02.1.adm
index c637d87..2a8a531 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-02/explain_object_constructor-02.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-02/explain_object_constructor-02.1.adm
@@ -2,7 +2,7 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    assign [$$2] <- [{ a: "1" }]
+    assign [$$2] <- [{ "a": "1" }]
     -- ASSIGN  |UNPARTITIONED|
       empty-tuple-source
-      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.adm
index f0fad1e..4c4d467 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.adm
@@ -1,22 +1,22 @@
-distribute result [$$17]
+distribute result [$$18]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$17])
+    project ([$$18])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$17] <- [{"display_url": $$20}]
+      assign [$$18] <- [{"display_url": $$21}]
       -- ASSIGN  |PARTITIONED|
-        project ([$$20])
+        project ([$$21])
         -- STREAM_PROJECT  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
-            order (ASC, $$19)
-            -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+            order (ASC, $$20)
+            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$20, $$19])
+                project ([$$21, $$20])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$20, $$19] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url"), $$p.getField("id")]
+                  assign [$$21, $$20] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url"), $$p.getField("id")]
                   -- ASSIGN  |PARTITIONED|
                     exchange
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.adm
index 2a24b09..bed813c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.adm
@@ -1,22 +1,22 @@
-distribute result [$$17]
+distribute result [$$18]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$17])
+    project ([$$18])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$17] <- [{"display_url": $$20}]
+      assign [$$18] <- [{"display_url": $$21}]
       -- ASSIGN  |PARTITIONED|
-        project ([$$20])
+        project ([$$21])
         -- STREAM_PROJECT  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
-            order (ASC, $$19)
-            -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+            order (ASC, $$20)
+            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$20, $$19])
+                project ([$$21, $$20])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$20, $$19] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url"), $$p.getField("id")]
+                  assign [$$21, $$20] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url"), $$p.getField("id")]
                   -- ASSIGN  |PARTITIONED|
                     exchange
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.adm
index 1ed4eed..f8b141f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.adm
@@ -1,28 +1,28 @@
-distribute result [$$21]
+distribute result [$$22]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$21])
+    project ([$$22])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$21] <- [{"display_url": $$25}]
+      assign [$$22] <- [{"display_url": $$26}]
       -- ASSIGN  |PARTITIONED|
-        project ([$$25])
+        project ([$$26])
         -- STREAM_PROJECT  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-            order (ASC, $$24)
-            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+            order (ASC, $$25)
+            -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$25, $$24])
+                project ([$$26, $$25])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$25] <- [array-star($$23).getField("display_url")]
+                  assign [$$26] <- [array-star($$24).getField("display_url")]
                   -- ASSIGN  |PARTITIONED|
-                    select (not(is-missing($$23)))
+                    select (not(is-missing($$24)))
                     -- STREAM_SELECT  |PARTITIONED|
-                      project ([$$23, $$24])
+                      project ([$$24, $$25])
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$23, $$24] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")]
+                        assign [$$24, $$25] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")]
                         -- ASSIGN  |PARTITIONED|
                           exchange
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.adm
index 2302d35..037d71b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.adm
@@ -1,28 +1,28 @@
-distribute result [$$21]
+distribute result [$$22]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$21])
+    project ([$$22])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$21] <- [{"display_url": $$25}]
+      assign [$$22] <- [{"display_url": $$26}]
       -- ASSIGN  |PARTITIONED|
-        project ([$$25])
+        project ([$$26])
         -- STREAM_PROJECT  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
-            order (ASC, $$24)
-            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
+            order (ASC, $$25)
+            -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$25, $$24])
+                project ([$$26, $$25])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$25] <- [array-star($$23).getField("display_url")]
+                  assign [$$26] <- [array-star($$24).getField("display_url")]
                   -- ASSIGN  |PARTITIONED|
-                    select (not(is-missing($$23)))
+                    select (not(is-missing($$24)))
                     -- STREAM_SELECT  |PARTITIONED|
-                      project ([$$23, $$24])
+                      project ([$$24, $$25])
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$23, $$24] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")]
+                        assign [$$24, $$25] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")]
                         -- ASSIGN  |PARTITIONED|
                           exchange
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.adm
index a9ee166..1a50cc8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.adm
@@ -1,26 +1,26 @@
-distribute result [$$26]
+distribute result [$$28]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$26])
+    project ([$$28])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$26] <- [{"display_url": $$urls.getField("display_url")}]
+      assign [$$28] <- [{"display_url": $$urls.getField("display_url")}]
       -- ASSIGN  |PARTITIONED|
         project ([$$urls])
         -- STREAM_PROJECT  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$29(ASC) ]  |PARTITIONED|
-            order (ASC, $$29)
-            -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$31(ASC) ]  |PARTITIONED|
+            order (ASC, $$31)
+            -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$urls, $$29])
+                project ([$$urls, $$31])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  unnest $$urls <- scan-collection($$28)
+                  unnest $$urls <- scan-collection($$30)
                   -- UNNEST  |PARTITIONED|
-                    project ([$$28, $$29])
+                    project ([$$30, $$31])
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$28, $$29] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")]
+                      assign [$$30, $$31] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")]
                       -- ASSIGN  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.adm
index a29ef6c..6b08be7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.adm
@@ -1,26 +1,26 @@
-distribute result [$$26]
+distribute result [$$28]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$26])
+    project ([$$28])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$26] <- [{"display_url": $$urls.getField("display_url")}]
+      assign [$$28] <- [{"display_url": $$urls.getField("display_url")}]
       -- ASSIGN  |PARTITIONED|
         project ([$$urls])
         -- STREAM_PROJECT  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$29(ASC) ]  |PARTITIONED|
-            order (ASC, $$29)
-            -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$31(ASC) ]  |PARTITIONED|
+            order (ASC, $$31)
+            -- STABLE_SORT [$$31(ASC)]  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$urls, $$29])
+                project ([$$urls, $$31])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  unnest $$urls <- scan-collection($$28)
+                  unnest $$urls <- scan-collection($$30)
                   -- UNNEST  |PARTITIONED|
-                    project ([$$28, $$29])
+                    project ([$$30, $$31])
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$28, $$29] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")]
+                      assign [$$30, $$31] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")]
                       -- ASSIGN  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.adm
index de214d0..6f31228 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.adm
@@ -1,33 +1,33 @@
-distribute result [$$46]
+distribute result [$$49]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    aggregate [$$46] <- [agg-sql-sum($$52)]
+    aggregate [$$49] <- [agg-sql-sum($$55)]
     -- AGGREGATE  |UNPARTITIONED|
       exchange
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        aggregate [$$52] <- [agg-sql-count(1)]
+        aggregate [$$55] <- [agg-sql-count(1)]
         -- AGGREGATE  |PARTITIONED|
-          select ($$39)
+          select ($$42)
           -- STREAM_SELECT  |PARTITIONED|
-            project ([$$39])
+            project ([$$42])
             -- STREAM_PROJECT  |PARTITIONED|
               subplan {
-                        aggregate [$$39] <- [empty-stream()]
+                        aggregate [$$42] <- [empty-stream()]
                         -- AGGREGATE  |LOCAL|
-                          select (not(if-missing-or-null(eq($$48, "string"), FALSE)))
+                          select (not(if-missing-or-null(eq($$51, "string"), false)))
                           -- STREAM_SELECT  |LOCAL|
-                            assign [$$48] <- [$$ht.getField("display_url")]
+                            assign [$$51] <- [$$ht.getField("display_url")]
                             -- ASSIGN  |LOCAL|
-                              unnest $$ht <- scan-collection($$47)
+                              unnest $$ht <- scan-collection($$50)
                               -- UNNEST  |LOCAL|
                                 nested tuple source
                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                      }
               -- SUBPLAN  |PARTITIONED|
-                project ([$$47])
+                project ([$$50])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$47] <- [$$p.getField("entities").getField("urls")]
+                  assign [$$50] <- [$$p.getField("entities").getField("urls")]
                   -- ASSIGN  |PARTITIONED|
                     exchange
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.adm
index 9f122c6..4981e71 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.adm
@@ -1,33 +1,33 @@
-distribute result [$$46]
+distribute result [$$49]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    aggregate [$$46] <- [agg-sql-sum($$52)]
+    aggregate [$$49] <- [agg-sql-sum($$55)]
     -- AGGREGATE  |UNPARTITIONED|
       exchange
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        aggregate [$$52] <- [agg-sql-count(1)]
+        aggregate [$$55] <- [agg-sql-count(1)]
         -- AGGREGATE  |PARTITIONED|
-          select ($$39)
+          select ($$42)
           -- STREAM_SELECT  |PARTITIONED|
-            project ([$$39])
+            project ([$$42])
             -- STREAM_PROJECT  |PARTITIONED|
               subplan {
-                        aggregate [$$39] <- [empty-stream()]
+                        aggregate [$$42] <- [empty-stream()]
                         -- AGGREGATE  |LOCAL|
-                          select (not(if-missing-or-null(eq($$48, "string"), FALSE)))
+                          select (not(if-missing-or-null(eq($$51, "string"), false)))
                           -- STREAM_SELECT  |LOCAL|
-                            assign [$$48] <- [$$ht.getField("display_url")]
+                            assign [$$51] <- [$$ht.getField("display_url")]
                             -- ASSIGN  |LOCAL|
-                              unnest $$ht <- scan-collection($$47)
+                              unnest $$ht <- scan-collection($$50)
                               -- UNNEST  |LOCAL|
                                 nested tuple source
                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                      }
               -- SUBPLAN  |PARTITIONED|
-                project ([$$47])
+                project ([$$50])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$47] <- [$$p.getField("entities").getField("urls")]
+                  assign [$$50] <- [$$p.getField("entities").getField("urls")]
                   -- ASSIGN  |PARTITIONED|
                     exchange
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.adm
index 37849ca..d3da104 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.adm
@@ -1,22 +1,22 @@
-distribute result [$$18]
+distribute result [$$19]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$18])
+    project ([$$19])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$18] <- [get-item(get-item(get-item($$21, 0), 0), 0)]
+      assign [$$19] <- [get-item(get-item(get-item($$22, 0), 0), 0)]
       -- ASSIGN  |PARTITIONED|
-        project ([$$21])
+        project ([$$22])
         -- STREAM_PROJECT  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-            order (ASC, $$20)
-            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+            order (ASC, $$21)
+            -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$21, $$20])
+                project ([$$22, $$21])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$21, $$20] <- [$$p.getField("place").getField("bounding_box").getField("coordinates"), $$p.getField("id")]
+                  assign [$$22, $$21] <- [$$p.getField("place").getField("bounding_box").getField("coordinates"), $$p.getField("id")]
                   -- ASSIGN  |PARTITIONED|
                     exchange
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.adm
index ee3a128..451ffec 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.adm
@@ -1,22 +1,22 @@
-distribute result [$$18]
+distribute result [$$19]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$18])
+    project ([$$19])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$18] <- [get-item(get-item(get-item($$21, 0), 0), 0)]
+      assign [$$19] <- [get-item(get-item(get-item($$22, 0), 0), 0)]
       -- ASSIGN  |PARTITIONED|
-        project ([$$21])
+        project ([$$22])
         -- STREAM_PROJECT  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-            order (ASC, $$20)
-            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+            order (ASC, $$21)
+            -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$21, $$20])
+                project ([$$22, $$21])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$21, $$20] <- [$$p.getField("place").getField("bounding_box").getField("coordinates"), $$p.getField("id")]
+                  assign [$$22, $$21] <- [$$p.getField("place").getField("bounding_box").getField("coordinates"), $$p.getField("id")]
                   -- ASSIGN  |PARTITIONED|
                     exchange
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.03.adm
index 5260a0a..7bc9fce 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.03.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.03.adm
@@ -5,12 +5,12 @@
     project ([$$p])
     -- STREAM_PROJECT  |PARTITIONED|
       exchange
-      -- SORT_MERGE_EXCHANGE [$$13(ASC) ]  |PARTITIONED|
-        order (ASC, $$13)
-        -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$14(ASC) ]  |PARTITIONED|
+        order (ASC, $$14)
+        -- STABLE_SORT [$$14(ASC)]  |PARTITIONED|
           exchange
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            assign [$$13] <- [$$p.getField("id")]
+            assign [$$14] <- [$$p.getField("id")]
             -- ASSIGN  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.05.adm
index 5260a0a..7bc9fce 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.05.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.05.adm
@@ -5,12 +5,12 @@
     project ([$$p])
     -- STREAM_PROJECT  |PARTITIONED|
       exchange
-      -- SORT_MERGE_EXCHANGE [$$13(ASC) ]  |PARTITIONED|
-        order (ASC, $$13)
-        -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$14(ASC) ]  |PARTITIONED|
+        order (ASC, $$14)
+        -- STABLE_SORT [$$14(ASC)]  |PARTITIONED|
           exchange
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            assign [$$13] <- [$$p.getField("id")]
+            assign [$$14] <- [$$p.getField("id")]
             -- ASSIGN  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.adm
index 48d64c1..73309ac 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.adm
@@ -1,26 +1,26 @@
-distribute result [$$28]
+distribute result [$$30]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$28])
+    project ([$$30])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$28] <- [{"p1": $$p1, "id": $$30}]
+      assign [$$30] <- [{"p1": $$p1, "id": $$32}]
       -- ASSIGN  |PARTITIONED|
         exchange
-        -- SORT_MERGE_EXCHANGE [$$30(ASC) ]  |PARTITIONED|
-          order (ASC, $$30)
-          -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$32(ASC) ]  |PARTITIONED|
+          order (ASC, $$32)
+          -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
             exchange
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$p1, $$30])
+              project ([$$p1, $$32])
               -- STREAM_PROJECT  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  join (eq($$31, $$30))
-                  -- HYBRID_HASH_JOIN [$$31][$$30]  |PARTITIONED|
+                  join (eq($$33, $$32))
+                  -- HYBRID_HASH_JOIN [$$33][$$32]  |PARTITIONED|
                     exchange
-                    -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
-                      assign [$$31] <- [$$p1.getField("id")]
+                    -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
+                      assign [$$33] <- [$$p1.getField("id")]
                       -- ASSIGN  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -31,10 +31,10 @@
                               empty-tuple-source
                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                     exchange
-                    -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
-                      project ([$$30])
+                    -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+                      project ([$$32])
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$30] <- [$$p2.getField("id")]
+                        assign [$$32] <- [$$p2.getField("id")]
                         -- ASSIGN  |PARTITIONED|
                           exchange
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.adm
index 2e7bd2b..f296c01 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.adm
@@ -1,26 +1,26 @@
-distribute result [$$28]
+distribute result [$$30]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$28])
+    project ([$$30])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$28] <- [{"p1": $$p1, "id": $$30}]
+      assign [$$30] <- [{"p1": $$p1, "id": $$32}]
       -- ASSIGN  |PARTITIONED|
         exchange
-        -- SORT_MERGE_EXCHANGE [$$30(ASC) ]  |PARTITIONED|
-          order (ASC, $$30)
-          -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$32(ASC) ]  |PARTITIONED|
+          order (ASC, $$32)
+          -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
             exchange
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              project ([$$p1, $$30])
+              project ([$$p1, $$32])
               -- STREAM_PROJECT  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  join (eq($$31, $$30))
-                  -- HYBRID_HASH_JOIN [$$31][$$30]  |PARTITIONED|
+                  join (eq($$33, $$32))
+                  -- HYBRID_HASH_JOIN [$$33][$$32]  |PARTITIONED|
                     exchange
-                    -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
-                      assign [$$31] <- [$$p1.getField("id")]
+                    -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
+                      assign [$$33] <- [$$p1.getField("id")]
                       -- ASSIGN  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -31,10 +31,10 @@
                               empty-tuple-source
                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                     exchange
-                    -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
-                      project ([$$30])
+                    -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+                      project ([$$32])
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$30] <- [$$p2.getField("id")]
+                        assign [$$32] <- [$$p2.getField("id")]
                         -- ASSIGN  |PARTITIONED|
                           exchange
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.adm
index 8eb02fe..b6835ed 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.adm
@@ -1,30 +1,30 @@
-distribute result [$$29]
+distribute result [$$31]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$29])
+    project ([$$31])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$29] <- [{"age": $$34, "name": $$35}]
+      assign [$$31] <- [{"age": $$36, "name": $$37}]
       -- ASSIGN  |PARTITIONED|
-        project ([$$34, $$35])
+        project ([$$36, $$37])
         -- STREAM_PROJECT  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$32(ASC) ]  |PARTITIONED|
-            order (ASC, $$32)
-            -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$34(ASC) ]  |PARTITIONED|
+            order (ASC, $$34)
+            -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$34, $$35, $$32])
+                project ([$$36, $$37, $$34])
                 -- STREAM_PROJECT  |PARTITIONED|
                   exchange
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    join (eq($$31, $$32))
-                    -- HYBRID_HASH_JOIN [$$31][$$32]  |PARTITIONED|
+                    join (eq($$33, $$34))
+                    -- HYBRID_HASH_JOIN [$$33][$$34]  |PARTITIONED|
                       exchange
-                      -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
-                        project ([$$34, $$31])
+                      -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
+                        project ([$$36, $$33])
                         -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$34, $$31] <- [$$p1.getField("age"), $$p1.getField("id")]
+                          assign [$$36, $$33] <- [$$p1.getField("age"), $$p1.getField("id")]
                           -- ASSIGN  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -35,10 +35,10 @@
                                   empty-tuple-source
                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                       exchange
-                      -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
-                        project ([$$35, $$32])
+                      -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+                        project ([$$37, $$34])
                         -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$35, $$32] <- [$$p2.getField("name"), $$p2.getField("id")]
+                          assign [$$37, $$34] <- [$$p2.getField("name"), $$p2.getField("id")]
                           -- ASSIGN  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.adm
index 12e1b6b..c0ddc2b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.adm
@@ -1,30 +1,30 @@
-distribute result [$$29]
+distribute result [$$31]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$29])
+    project ([$$31])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$29] <- [{"age": $$34, "name": $$35}]
+      assign [$$31] <- [{"age": $$36, "name": $$37}]
       -- ASSIGN  |PARTITIONED|
-        project ([$$34, $$35])
+        project ([$$36, $$37])
         -- STREAM_PROJECT  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$32(ASC) ]  |PARTITIONED|
-            order (ASC, $$32)
-            -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$34(ASC) ]  |PARTITIONED|
+            order (ASC, $$34)
+            -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$34, $$35, $$32])
+                project ([$$36, $$37, $$34])
                 -- STREAM_PROJECT  |PARTITIONED|
                   exchange
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    join (eq($$31, $$32))
-                    -- HYBRID_HASH_JOIN [$$31][$$32]  |PARTITIONED|
+                    join (eq($$33, $$34))
+                    -- HYBRID_HASH_JOIN [$$33][$$34]  |PARTITIONED|
                       exchange
-                      -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
-                        project ([$$34, $$31])
+                      -- HASH_PARTITION_EXCHANGE [$$33]  |PARTITIONED|
+                        project ([$$36, $$33])
                         -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$34, $$31] <- [$$p1.getField("age"), $$p1.getField("id")]
+                          assign [$$36, $$33] <- [$$p1.getField("age"), $$p1.getField("id")]
                           -- ASSIGN  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -35,10 +35,10 @@
                                   empty-tuple-source
                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                       exchange
-                      -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
-                        project ([$$35, $$32])
+                      -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
+                        project ([$$37, $$34])
                         -- STREAM_PROJECT  |PARTITIONED|
-                          assign [$$35, $$32] <- [$$p2.getField("name"), $$p2.getField("id")]
+                          assign [$$37, $$34] <- [$$p2.getField("name"), $$p2.getField("id")]
                           -- ASSIGN  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.adm
index dc8c103..c5fdaca 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.adm
@@ -1,26 +1,26 @@
-distribute result [$$17]
+distribute result [$$18]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$17])
+    project ([$$18])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$17] <- [{"id": $$21, "name": $$22}]
+      assign [$$18] <- [{"id": $$22, "name": $$23}]
       -- ASSIGN  |PARTITIONED|
-        project ([$$21, $$22])
+        project ([$$22, $$23])
         -- STREAM_PROJECT  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-            order (ASC, $$20)
-            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+            order (ASC, $$21)
+            -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$21, $$22, $$20])
+                project ([$$22, $$23, $$21])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")]
+                  assign [$$23, $$22] <- [$$20.getField("name"), $$20.getField("id")]
                   -- ASSIGN  |PARTITIONED|
-                    project ([$$19, $$20])
+                    project ([$$20, $$21])
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")]
+                      assign [$$20, $$21] <- [$$p.getField("user"), $$p.getField("id")]
                       -- ASSIGN  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.adm
index 1ffc9a7..89b9f29 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.adm
@@ -1,26 +1,26 @@
-distribute result [$$17]
+distribute result [$$18]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$17])
+    project ([$$18])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$17] <- [{"id": $$21, "name": $$22}]
+      assign [$$18] <- [{"id": $$22, "name": $$23}]
       -- ASSIGN  |PARTITIONED|
-        project ([$$21, $$22])
+        project ([$$22, $$23])
         -- STREAM_PROJECT  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-            order (ASC, $$20)
-            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+            order (ASC, $$21)
+            -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$21, $$22, $$20])
+                project ([$$22, $$23, $$21])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")]
+                  assign [$$23, $$22] <- [$$20.getField("name"), $$20.getField("id")]
                   -- ASSIGN  |PARTITIONED|
-                    project ([$$19, $$20])
+                    project ([$$20, $$21])
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")]
+                      assign [$$20, $$21] <- [$$p.getField("user"), $$p.getField("id")]
                       -- ASSIGN  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.adm
index 229e53a..16ef812 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.adm
@@ -1,12 +1,12 @@
-distribute result [$$31]
+distribute result [$$33]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    aggregate [$$31] <- [agg-sql-sum($$32)]
+    aggregate [$$33] <- [agg-sql-sum($$34)]
     -- AGGREGATE  |UNPARTITIONED|
       exchange
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        aggregate [$$32] <- [agg-sql-count(1)]
+        aggregate [$$34] <- [agg-sql-count(1)]
         -- AGGREGATE  |PARTITIONED|
           exchange
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.adm
index 6937d90..56d31b1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.adm
@@ -1,12 +1,12 @@
-distribute result [$$31]
+distribute result [$$33]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    aggregate [$$31] <- [agg-sql-sum($$32)]
+    aggregate [$$33] <- [agg-sql-sum($$34)]
     -- AGGREGATE  |UNPARTITIONED|
       exchange
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        aggregate [$$32] <- [agg-sql-count(1)]
+        aggregate [$$34] <- [agg-sql-count(1)]
         -- AGGREGATE  |PARTITIONED|
           exchange
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.adm
index 1ffc9a7..89b9f29 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.adm
@@ -1,26 +1,26 @@
-distribute result [$$17]
+distribute result [$$18]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$17])
+    project ([$$18])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$17] <- [{"id": $$21, "name": $$22}]
+      assign [$$18] <- [{"id": $$22, "name": $$23}]
       -- ASSIGN  |PARTITIONED|
-        project ([$$21, $$22])
+        project ([$$22, $$23])
         -- STREAM_PROJECT  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-            order (ASC, $$20)
-            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+            order (ASC, $$21)
+            -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$21, $$22, $$20])
+                project ([$$22, $$23, $$21])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")]
+                  assign [$$23, $$22] <- [$$20.getField("name"), $$20.getField("id")]
                   -- ASSIGN  |PARTITIONED|
-                    project ([$$19, $$20])
+                    project ([$$20, $$21])
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")]
+                      assign [$$20, $$21] <- [$$p.getField("user"), $$p.getField("id")]
                       -- ASSIGN  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.adm
index 4a602ef..95c26a0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.adm
@@ -1,24 +1,24 @@
-distribute result [$$20]
+distribute result [$$21]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$20])
+    project ([$$21])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$20] <- [switch-case(TRUE, is-array($$22), $$24, $$25)]
+      assign [$$21] <- [switch-case(true, is-array($$23), $$25, $$26)]
       -- ASSIGN  |PARTITIONED|
-        project ([$$22, $$24, $$25])
+        project ([$$23, $$25, $$26])
         -- STREAM_PROJECT  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-            order (ASC, $$23)
-            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+            order (ASC, $$24)
+            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                assign [$$24, $$25] <- [array-star($$22).getField("text"), $$22.getField("text")]
+                assign [$$25, $$26] <- [array-star($$23).getField("text"), $$23.getField("text")]
                 -- ASSIGN  |PARTITIONED|
-                  project ([$$22, $$23])
+                  project ([$$23, $$24])
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$22, $$23] <- [$$p.getField("arrayOrObject"), $$p.getField("id")]
+                    assign [$$23, $$24] <- [$$p.getField("arrayOrObject"), $$p.getField("id")]
                     -- ASSIGN  |PARTITIONED|
                       exchange
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.adm
index 56dd2ee..daa64a6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.adm
@@ -1,24 +1,24 @@
-distribute result [$$20]
+distribute result [$$21]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$20])
+    project ([$$21])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$20] <- [switch-case(TRUE, is-array($$22), $$24, $$25)]
+      assign [$$21] <- [switch-case(true, is-array($$23), $$25, $$26)]
       -- ASSIGN  |PARTITIONED|
-        project ([$$22, $$24, $$25])
+        project ([$$23, $$25, $$26])
         -- STREAM_PROJECT  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
-            order (ASC, $$23)
-            -- STABLE_SORT [$$23(ASC)]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+            order (ASC, $$24)
+            -- STABLE_SORT [$$24(ASC)]  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                assign [$$24, $$25] <- [array-star($$22).getField("text"), $$22.getField("text")]
+                assign [$$25, $$26] <- [array-star($$23).getField("text"), $$23.getField("text")]
                 -- ASSIGN  |PARTITIONED|
-                  project ([$$22, $$23])
+                  project ([$$23, $$24])
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$22, $$23] <- [$$p.getField("arrayOrObject"), $$p.getField("id")]
+                    assign [$$23, $$24] <- [$$p.getField("arrayOrObject"), $$p.getField("id")]
                     -- ASSIGN  |PARTITIONED|
                       exchange
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.3.adm
index 365218f..a1e3eb9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.3.adm
@@ -1,18 +1,18 @@
-distribute result [$$16]
+distribute result [$$17]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$16])
+    project ([$$17])
     -- STREAM_PROJECT  |PARTITIONED|
       exchange
-      -- SORT_MERGE_EXCHANGE [$$18(ASC) ]  |PARTITIONED|
-        order (ASC, $$18)
-        -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$19(ASC) ]  |PARTITIONED|
+        order (ASC, $$19)
+        -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
           exchange
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            project ([$$16, $$18])
+            project ([$$17, $$19])
             -- STREAM_PROJECT  |PARTITIONED|
-              assign [$$16, $$18] <- [object-concat($$p.getField("coordinates"), $$p.getField("user")).getField("name"), $$p.getField("id")]
+              assign [$$17, $$19] <- [object-concat($$p.getField("coordinates"), $$p.getField("user")).getField("name"), $$p.getField("id")]
               -- ASSIGN  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.5.adm
index 4533f63..04e521e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.5.adm
@@ -1,22 +1,22 @@
-distribute result [$$15]
+distribute result [$$16]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$15])
+    project ([$$16])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$15] <- [object-concat($$18, $$19)]
+      assign [$$16] <- [object-concat($$19, $$20)]
       -- ASSIGN  |PARTITIONED|
-        project ([$$18, $$19])
+        project ([$$19, $$20])
         -- STREAM_PROJECT  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$17(ASC) ]  |PARTITIONED|
-            order (ASC, $$17)
-            -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$18(ASC) ]  |PARTITIONED|
+            order (ASC, $$18)
+            -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                project ([$$18, $$19, $$17])
+                project ([$$19, $$20, $$18])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$19, $$18, $$17] <- [$$p.getField("user"), $$p.getField("coordinates"), $$p.getField("id")]
+                  assign [$$20, $$19, $$18] <- [$$p.getField("user"), $$p.getField("coordinates"), $$p.getField("id")]
                   -- ASSIGN  |PARTITIONED|
                     exchange
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.adm
index b020ac9..50f1ba8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.adm
@@ -1,30 +1,30 @@
-distribute result [$$48]
+distribute result [$$51]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$48])
+    project ([$$51])
     -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$48] <- [{"$1": $$51}]
+      assign [$$51] <- [{"$1": $$54}]
       -- ASSIGN  |UNPARTITIONED|
-        aggregate [$$51] <- [agg-global-sql-sum($$53)]
+        aggregate [$$54] <- [agg-global-sql-sum($$56)]
         -- AGGREGATE  |UNPARTITIONED|
           exchange
           -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-            aggregate [$$53] <- [agg-local-sql-sum($$46)]
+            aggregate [$$56] <- [agg-local-sql-sum($$49)]
             -- AGGREGATE  |PARTITIONED|
-              project ([$$46])
+              project ([$$49])
               -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$46] <- [object-length($$p1)]
+                assign [$$49] <- [object-length($$p1)]
                 -- ASSIGN  |PARTITIONED|
                   project ([$$p1])
                   -- STREAM_PROJECT  |PARTITIONED|
                     exchange
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      join (eq($$49, $$50))
-                      -- HYBRID_HASH_JOIN [$$50][$$49]  |PARTITIONED|
+                      join (eq($$52, $$53))
+                      -- HYBRID_HASH_JOIN [$$53][$$52]  |PARTITIONED|
                         exchange
-                        -- HASH_PARTITION_EXCHANGE [$$50]  |PARTITIONED|
-                          assign [$$50] <- [$$p1.getField("id")]
+                        -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                          assign [$$53] <- [$$p1.getField("id")]
                           -- ASSIGN  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -35,10 +35,10 @@
                                   empty-tuple-source
                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                         exchange
-                        -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
-                          project ([$$49])
+                        -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+                          project ([$$52])
                           -- STREAM_PROJECT  |PARTITIONED|
-                            assign [$$49] <- [$$p2.getField("id")]
+                            assign [$$52] <- [$$p2.getField("id")]
                             -- ASSIGN  |PARTITIONED|
                               exchange
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.adm
index d22cd15..de0f762 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.adm
@@ -1,42 +1,42 @@
-distribute result [$$65]
+distribute result [$$69]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$65])
+    project ([$$69])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$65] <- [{"text": $$text, "$1": $$68}]
+      assign [$$69] <- [{"text": $$text, "$1": $$72}]
       -- ASSIGN  |PARTITIONED|
         exchange
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$$text := $$76]) decor ([]) {
-                    aggregate [$$68] <- [agg-global-sql-sum($$75)]
+          group by ([$$text := $$80]) decor ([]) {
+                    aggregate [$$72] <- [agg-global-sql-sum($$79)]
                     -- AGGREGATE  |LOCAL|
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                  }
-          -- SORT_GROUP_BY[$$76]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$80]  |PARTITIONED|
             exchange
-            -- HASH_PARTITION_EXCHANGE [$$76]  |PARTITIONED|
-              group by ([$$76 := $$67]) decor ([]) {
-                        aggregate [$$75] <- [agg-local-sql-sum(array-distinct($$70))]
+            -- HASH_PARTITION_EXCHANGE [$$80]  |PARTITIONED|
+              group by ([$$80 := $$71]) decor ([]) {
+                        aggregate [$$79] <- [agg-local-sql-sum(array-distinct($$74))]
                         -- AGGREGATE  |LOCAL|
                           nested tuple source
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                      }
-              -- SORT_GROUP_BY[$$67]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$71]  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  select (eq(lowercase($$67), "string"))
+                  select (eq(lowercase($$71), "string"))
                   -- STREAM_SELECT  |PARTITIONED|
-                    project ([$$70, $$67])
+                    project ([$$74, $$71])
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$67] <- [$$ht.getField("text")]
+                      assign [$$71] <- [$$ht.getField("text")]
                       -- ASSIGN  |PARTITIONED|
-                        unnest $$ht <- scan-collection($$70)
+                        unnest $$ht <- scan-collection($$74)
                         -- UNNEST  |PARTITIONED|
-                          project ([$$70])
+                          project ([$$74])
                           -- STREAM_PROJECT  |PARTITIONED|
-                            assign [$$70] <- [$$p1.getField("entities").getField("hashtags")]
+                            assign [$$74] <- [$$p1.getField("entities").getField("hashtags")]
                             -- ASSIGN  |PARTITIONED|
                               select (gt($$p1.getField("id"), 10))
                               -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.adm
index 8ad73f9..8f6bdd8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.adm
@@ -1,44 +1,44 @@
-distribute result [$$64]
+distribute result [$$68]
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    project ([$$64])
+    project ([$$68])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$64] <- [{"text": $$text, "$1": $$67}]
+      assign [$$68] <- [{"text": $$text, "$1": $$71}]
       -- ASSIGN  |PARTITIONED|
         exchange
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$$text := $$74]) decor ([]) {
-                    aggregate [$$67] <- [agg-global-sql-sum($$73)]
+          group by ([$$text := $$78]) decor ([]) {
+                    aggregate [$$71] <- [agg-global-sql-sum($$77)]
                     -- AGGREGATE  |LOCAL|
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                  }
-          -- SORT_GROUP_BY[$$74]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$78]  |PARTITIONED|
             exchange
-            -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
-              group by ([$$74 := $$66]) decor ([]) {
-                        aggregate [$$73] <- [agg-local-sql-sum(sql-sum($$70))]
+            -- HASH_PARTITION_EXCHANGE [$$78]  |PARTITIONED|
+              group by ([$$78 := $$70]) decor ([]) {
+                        aggregate [$$77] <- [agg-local-sql-sum(sql-sum($$74))]
                         -- AGGREGATE  |LOCAL|
                           nested tuple source
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                      }
-              -- SORT_GROUP_BY[$$66]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$70]  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  select (eq(lowercase($$66), "string"))
+                  select (eq(lowercase($$70), "string"))
                   -- STREAM_SELECT  |PARTITIONED|
-                    project ([$$70, $$66])
+                    project ([$$74, $$70])
                     -- STREAM_PROJECT  |PARTITIONED|
-                      assign [$$70, $$66] <- [$$ht.getField("indices"), $$ht.getField("text")]
+                      assign [$$74, $$70] <- [$$ht.getField("indices"), $$ht.getField("text")]
                       -- ASSIGN  |PARTITIONED|
                         project ([$$ht])
                         -- STREAM_PROJECT  |PARTITIONED|
-                          unnest $$ht <- scan-collection($$69)
+                          unnest $$ht <- scan-collection($$73)
                           -- UNNEST  |PARTITIONED|
-                            project ([$$69])
+                            project ([$$73])
                             -- STREAM_PROJECT  |PARTITIONED|
-                              assign [$$69] <- [$$p1.getField("entities").getField("hashtags")]
+                              assign [$$73] <- [$$p1.getField("entities").getField("hashtags")]
                               -- ASSIGN  |PARTITIONED|
                                 select (gt($$p1.getField("id"), 10))
                                 -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.adm
index cac9db7..80c62d9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.adm
@@ -1,4 +1,4 @@
-distribute result [$$21]
+distribute result [$$22]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
@@ -6,15 +6,15 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        project ([$$21])
+        project ([$$22])
         -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$21] <- [{"display_url": get-item($$22, 0).getField("display_url")}]
+          assign [$$22] <- [{"display_url": get-item($$23, 0).getField("display_url")}]
           -- ASSIGN  |PARTITIONED|
             limit 10
             -- STREAM_LIMIT  |PARTITIONED|
-              project ([$$22])
+              project ([$$23])
               -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$22] <- [$$p.getField("entities").getField("urls")]
+                assign [$$23] <- [$$p.getField("entities").getField("urls")]
                 -- ASSIGN  |PARTITIONED|
                   exchange
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.adm
index 30d5ac3..39dce27 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.adm
@@ -1,85 +1,85 @@
-distribute result [$$94]
+distribute result [$$101]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 10
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$94])
+      project ([$$101])
       -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$94] <- [{"uname": $$uname, "cnt": $$96}]
+        assign [$$101] <- [{"uname": $$uname, "cnt": $$103}]
         -- ASSIGN  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$96(DESC) ]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$103(DESC) ]  |PARTITIONED|
             limit 10
             -- STREAM_LIMIT  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (topK: 10) (DESC, $$96)
-                -- STABLE_SORT [topK: 10] [$$96(DESC)]  |PARTITIONED|
+                order (topK: 10) (DESC, $$103)
+                -- STABLE_SORT [topK: 10] [$$103(DESC)]  |PARTITIONED|
                   exchange
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    group by ([$$uname := $$107]) decor ([]) {
-                              aggregate [$$96] <- [agg-sql-sum($$106)]
+                    group by ([$$uname := $$114]) decor ([]) {
+                              aggregate [$$103] <- [agg-sql-sum($$113)]
                               -- AGGREGATE  |LOCAL|
                                 nested tuple source
                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                            }
-                    -- SORT_GROUP_BY[$$107]  |PARTITIONED|
+                    -- SORT_GROUP_BY[$$114]  |PARTITIONED|
                       exchange
-                      -- HASH_PARTITION_EXCHANGE [$$107]  |PARTITIONED|
-                        group by ([$$107 := $$95]) decor ([]) {
-                                  aggregate [$$106] <- [agg-sql-count(1)]
+                      -- HASH_PARTITION_EXCHANGE [$$114]  |PARTITIONED|
+                        group by ([$$114 := $$102]) decor ([]) {
+                                  aggregate [$$113] <- [agg-sql-count(1)]
                                   -- AGGREGATE  |LOCAL|
                                     nested tuple source
                                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                                }
-                        -- SORT_GROUP_BY[$$95]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$102]  |PARTITIONED|
                           exchange
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            project ([$$95])
+                            project ([$$102])
                             -- STREAM_PROJECT  |PARTITIONED|
-                              select ($$85)
+                              select ($$92)
                               -- STREAM_SELECT  |PARTITIONED|
-                                project ([$$85, $$95])
+                                project ([$$92, $$102])
                                 -- STREAM_PROJECT  |PARTITIONED|
                                   exchange
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    group by ([$$105 := $$103]) decor ([$$95]) {
-                                              aggregate [$$85] <- [non-empty-stream()]
+                                    group by ([$$112 := $$110]) decor ([$$102]) {
+                                              aggregate [$$92] <- [non-empty-stream()]
                                               -- AGGREGATE  |LOCAL|
-                                                select (not(is-missing($$104)))
+                                                select (not(is-missing($$111)))
                                                 -- STREAM_SELECT  |LOCAL|
                                                   nested tuple source
                                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                                            }
-                                    -- PRE_CLUSTERED_GROUP_BY[$$103]  |PARTITIONED|
+                                    -- PRE_CLUSTERED_GROUP_BY[$$110]  |PARTITIONED|
                                       exchange
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        order (ASC, $$103)
-                                        -- STABLE_SORT [$$103(ASC)]  |PARTITIONED|
+                                        order (ASC, $$110)
+                                        -- STABLE_SORT [$$110(ASC)]  |PARTITIONED|
                                           exchange
-                                          -- HASH_PARTITION_EXCHANGE [$$103]  |PARTITIONED|
-                                            project ([$$95, $$104, $$103])
+                                          -- HASH_PARTITION_EXCHANGE [$$110]  |PARTITIONED|
+                                            project ([$$102, $$111, $$110])
                                             -- STREAM_PROJECT  |PARTITIONED|
                                               exchange
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                left outer join (eq($$97, $$81))
-                                                -- HYBRID_HASH_JOIN [$$97][$$81]  |PARTITIONED|
+                                                left outer join (eq($$104, $$88))
+                                                -- HYBRID_HASH_JOIN [$$104][$$88]  |PARTITIONED|
                                                   exchange
-                                                  -- HASH_PARTITION_EXCHANGE [$$97]  |PARTITIONED|
-                                                    running-aggregate [$$103] <- [create-query-uid()]
+                                                  -- HASH_PARTITION_EXCHANGE [$$104]  |PARTITIONED|
+                                                    running-aggregate [$$110] <- [create-query-uid()]
                                                     -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                      project ([$$95, $$97])
+                                                      project ([$$102, $$104])
                                                       -- STREAM_PROJECT  |PARTITIONED|
-                                                        assign [$$97] <- [$$ht1.getField("text")]
+                                                        assign [$$104] <- [$$ht1.getField("text")]
                                                         -- ASSIGN  |PARTITIONED|
-                                                          project ([$$95, $$ht1])
+                                                          project ([$$102, $$ht1])
                                                           -- STREAM_PROJECT  |PARTITIONED|
-                                                            unnest $$ht1 <- scan-collection($$98)
+                                                            unnest $$ht1 <- scan-collection($$105)
                                                             -- UNNEST  |PARTITIONED|
-                                                              project ([$$98, $$95])
+                                                              project ([$$105, $$102])
                                                               -- STREAM_PROJECT  |PARTITIONED|
-                                                                assign [$$98, $$95] <- [$$p1.getField("entities").getField("hashtags"), $$p1.getField("user").getField("name")]
+                                                                assign [$$105, $$102] <- [$$p1.getField("entities").getField("hashtags"), $$p1.getField("user").getField("name")]
                                                                 -- ASSIGN  |PARTITIONED|
                                                                   exchange
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -90,18 +90,18 @@
                                                                         empty-tuple-source
                                                                         -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                                   exchange
-                                                  -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
-                                                    project ([$$104, $$81])
+                                                  -- HASH_PARTITION_EXCHANGE [$$88]  |PARTITIONED|
+                                                    project ([$$111, $$88])
                                                     -- STREAM_PROJECT  |PARTITIONED|
-                                                      assign [$$104, $$81] <- [TRUE, $$ht2.getField("text")]
+                                                      assign [$$111, $$88] <- [true, $$ht2.getField("text")]
                                                       -- ASSIGN  |PARTITIONED|
                                                         project ([$$ht2])
                                                         -- STREAM_PROJECT  |PARTITIONED|
-                                                          unnest $$ht2 <- scan-collection($$99)
+                                                          unnest $$ht2 <- scan-collection($$106)
                                                           -- UNNEST  |PARTITIONED|
-                                                            project ([$$99])
+                                                            project ([$$106])
                                                             -- STREAM_PROJECT  |PARTITIONED|
-                                                              assign [$$99] <- [$$p2.getField("entities").getField("hashtags")]
+                                                              assign [$$106] <- [$$p2.getField("entities").getField("hashtags")]
                                                               -- ASSIGN  |PARTITIONED|
                                                                 exchange
                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/query-with-limit-plan/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/query-with-limit-plan/result.001.adm
index c7927c0..9149be2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/query-with-limit-plan/result.001.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/query-with-limit-plan/result.001.adm
@@ -1,4 +1,4 @@
-distribute result [$$13]
+distribute result [$$14]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
@@ -6,9 +6,9 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        project ([$$13])
+        project ([$$14])
         -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$13] <- [{"test": $$test}]
+          assign [$$14] <- [{"test": $$test}]
           -- ASSIGN  |PARTITIONED|
             limit 3
             -- STREAM_LIMIT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/type_validation/type_validation.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/type_validation/type_validation.1.adm
index f164f97..121262f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/type_validation/type_validation.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/type_validation/type_validation.1.adm
@@ -1 +1 @@
-"907 9.07 \"907\" 9.07 TRUE date: { 2013-01-01 } datetime: { 1989-09-07T12:13:14.039 } "
+"907 9.07 \"907\" 9.07 true date: { 2013-01-01 } datetime: { 1989-09-07T12:13:14.039 } "
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/inner_right_corr/inner_right_corr.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/inner_right_corr/inner_right_corr.3.adm
new file mode 100644
index 0000000..2b2783f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/inner_right_corr/inner_right_corr.3.adm
@@ -0,0 +1,2 @@
+{ "a": [ 3 ] }
+{ "a": [ 1 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.3.adm
new file mode 100644
index 0000000..e9eb380
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.3.adm
@@ -0,0 +1 @@
+{ "count": 159800 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.4.adm
new file mode 100644
index 0000000..e9eb380
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.4.adm
@@ -0,0 +1 @@
+{ "count": 159800 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.5.adm
new file mode 100644
index 0000000..f3142ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj_partitioning_property/nlj_partitioning_property.5.adm
@@ -0,0 +1 @@
+{ "count": 1600000 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/offset_without_limit/offset_without_limit.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/offset_without_limit/offset_without_limit.6.adm
index 726ee49..f91dedf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/offset_without_limit/offset_without_limit.6.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/offset_without_limit/offset_without_limit.6.adm
@@ -1,16 +1,16 @@
-distribute result [$$15]
+distribute result [$$16]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit offset 98
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$15])
+      project ([$$16])
       -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$15] <- [{"id": $$17, "dblpid": $$paper.getField(1)}]
+        assign [$$16] <- [{"id": $$18, "dblpid": $$paper.getField(1)}]
         -- ASSIGN  |PARTITIONED|
           exchange
-          -- SORT_MERGE_EXCHANGE [$$17(ASC) ]  |PARTITIONED|
-            data-scan []<-[$$17, $$paper] <- test.DBLP1
+          -- SORT_MERGE_EXCHANGE [$$18(ASC) ]  |PARTITIONED|
+            data-scan []<-[$$18, $$paper] <- test.DBLP1
             -- DATASOURCE_SCAN  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan-select/push-limit-to-external-scan-select.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan-select/push-limit-to-external-scan-select.2.adm
index b38ed8b..cc319ee 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan-select/push-limit-to-external-scan-select.2.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan-select/push-limit-to-external-scan-select.2.adm
@@ -1,4 +1,4 @@
-distribute result [$$16]
+distribute result [$$17]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
@@ -8,9 +8,9 @@
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
         limit 5
         -- STREAM_LIMIT  |PARTITIONED|
-          project ([$$16])
+          project ([$$17])
           -- STREAM_PROJECT  |PARTITIONED|
-            assign [$$16] <- [$$t.getField(0)]
+            assign [$$17] <- [$$t.getField(0)]
             -- ASSIGN  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan/push-limit-to-external-scan.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan/push-limit-to-external-scan.2.adm
index 2c98237..611efaa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan/push-limit-to-external-scan.2.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-external-scan/push-limit-to-external-scan.2.adm
@@ -1,4 +1,4 @@
-distribute result [$$13]
+distribute result [$$14]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
@@ -6,9 +6,9 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        project ([$$13])
+        project ([$$14])
         -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$13] <- [$$t.getField(0)]
+          assign [$$14] <- [$$t.getField(0)]
           -- ASSIGN  |PARTITIONED|
             limit 5
             -- STREAM_LIMIT  |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.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
index 3543f5d..435dbb0 100644
--- 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
@@ -7,28 +7,28 @@
       project ([$$c])
       -- STREAM_PROJECT  |PARTITIONED|
         exchange
-        -- SORT_MERGE_EXCHANGE [$$17(ASC), $$18(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$18(ASC), $$19(ASC) ]  |PARTITIONED|
           limit 10
           -- STREAM_LIMIT  |PARTITIONED|
             exchange
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              unnest-map [$$17, $$18, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$23, $$24, 2, $$23, $$24, TRUE, TRUE, TRUE) condition (and(lt($$c.getField(2), 150), lt($$c.getField(5), 10000))) limit 10
+              unnest-map [$$18, $$19, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$24, $$25, 2, $$24, $$25, 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, $$23) (ASC, $$24)
-                  -- STABLE_SORT [$$23(ASC), $$24(ASC)]  |PARTITIONED|
+                  order (ASC, $$24) (ASC, $$25)
+                  -- STABLE_SORT [$$24(ASC), $$25(ASC)]  |PARTITIONED|
                     exchange
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      project ([$$23, $$24])
+                      project ([$$24, $$25])
                       -- STREAM_PROJECT  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          unnest-map [$$22, $$23, $$24] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$21, TRUE, FALSE, FALSE)
+                          unnest-map [$$23, $$24, $$25] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$22, true, false, false)
                           -- BTREE_SEARCH  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              assign [$$21] <- [150]
+                              assign [$$22] <- [150]
                               -- ASSIGN  |PARTITIONED|
                                 empty-tuple-source
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.adm
index 1f4ef66..4baeedc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.adm
@@ -1,40 +1,40 @@
-distribute result [$$19]
+distribute result [$$20]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 5
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$19])
+      project ([$$20])
       -- STREAM_PROJECT  |PARTITIONED|
         exchange
-        -- SORT_MERGE_EXCHANGE [$$21(ASC), $$22(ASC) ]  |PARTITIONED|
-          project ([$$21, $$22, $$19])
+        -- SORT_MERGE_EXCHANGE [$$22(ASC), $$23(ASC) ]  |PARTITIONED|
+          project ([$$22, $$23, $$20])
           -- STREAM_PROJECT  |PARTITIONED|
-            assign [$$19] <- [{"shipdate": substring($$c.getField(10), 0, 4), "suppkey": gt($$20, 0)}]
+            assign [$$20] <- [{"shipdate": substring($$c.getField(10), 0, 4), "suppkey": gt($$21, 0)}]
             -- ASSIGN  |PARTITIONED|
               limit 5
               -- STREAM_LIMIT  |PARTITIONED|
-                assign [$$20] <- [$$c.getField(2)]
+                assign [$$21] <- [$$c.getField(2)]
                 -- ASSIGN  |PARTITIONED|
                   exchange
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    unnest-map [$$21, $$22, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$26, $$27, 2, $$26, $$27, TRUE, TRUE, TRUE) condition (lt($$c.getField(2), 150)) limit 5
+                    unnest-map [$$22, $$23, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$27, $$28, 2, $$27, $$28, true, true, true) condition (lt($$c.getField(2), 150)) limit 5
                     -- BTREE_SEARCH  |PARTITIONED|
                       exchange
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        order (ASC, $$26) (ASC, $$27)
-                        -- STABLE_SORT [$$26(ASC), $$27(ASC)]  |PARTITIONED|
+                        order (ASC, $$27) (ASC, $$28)
+                        -- STABLE_SORT [$$27(ASC), $$28(ASC)]  |PARTITIONED|
                           exchange
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            project ([$$26, $$27])
+                            project ([$$27, $$28])
                             -- STREAM_PROJECT  |PARTITIONED|
                               exchange
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                unnest-map [$$25, $$26, $$27] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$24, TRUE, FALSE, FALSE)
+                                unnest-map [$$26, $$27, $$28] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$25, true, false, false)
                                 -- BTREE_SEARCH  |PARTITIONED|
                                   exchange
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    assign [$$24] <- [150]
+                                    assign [$$25] <- [150]
                                     -- ASSIGN  |PARTITIONED|
                                       empty-tuple-source
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
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
index a0a3c84..26aa844 100644
--- 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
@@ -7,28 +7,28 @@
       project ([$$c])
       -- STREAM_PROJECT  |PARTITIONED|
         exchange
-        -- SORT_MERGE_EXCHANGE [$$14(ASC), $$15(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$15(ASC), $$16(ASC) ]  |PARTITIONED|
           limit 10
           -- STREAM_LIMIT  |PARTITIONED|
             exchange
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              unnest-map [$$14, $$15, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$19, $$20, 2, $$19, $$20, TRUE, TRUE, TRUE) condition (lt($$c.getField(2), 150)) limit 10
+              unnest-map [$$15, $$16, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$20, $$21, 2, $$20, $$21, true, true, true) condition (lt($$c.getField(2), 150)) limit 10
               -- BTREE_SEARCH  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  order (ASC, $$19) (ASC, $$20)
-                  -- STABLE_SORT [$$19(ASC), $$20(ASC)]  |PARTITIONED|
+                  order (ASC, $$20) (ASC, $$21)
+                  -- STABLE_SORT [$$20(ASC), $$21(ASC)]  |PARTITIONED|
                     exchange
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      project ([$$19, $$20])
+                      project ([$$20, $$21])
                       -- STREAM_PROJECT  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          unnest-map [$$18, $$19, $$20] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$17, TRUE, FALSE, FALSE)
+                          unnest-map [$$19, $$20, $$21] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$18, true, false, false)
                           -- BTREE_SEARCH  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              assign [$$17] <- [150]
+                              assign [$$18] <- [150]
                               -- ASSIGN  |PARTITIONED|
                                 empty-tuple-source
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
index 44507f4..c1b44fb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
@@ -7,28 +7,28 @@
       project ([$$c])
       -- STREAM_PROJECT  |PARTITIONED|
         exchange
-        -- SORT_MERGE_EXCHANGE [$$16(ASC), $$17(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$17(ASC), $$18(ASC) ]  |PARTITIONED|
           limit 10
           -- STREAM_LIMIT  |PARTITIONED|
             exchange
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              unnest-map [$$16, $$17, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$21, $$22, 2, $$21, $$22, TRUE, TRUE, TRUE) condition (lt($$c.getField(2), 150)) limit 10
+              unnest-map [$$17, $$18, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$22, $$23, 2, $$22, $$23, true, true, true) condition (lt($$c.getField(2), 150)) limit 10
               -- BTREE_SEARCH  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  order (ASC, $$21) (ASC, $$22)
-                  -- STABLE_SORT [$$21(ASC), $$22(ASC)]  |PARTITIONED|
+                  order (ASC, $$22) (ASC, $$23)
+                  -- STABLE_SORT [$$22(ASC), $$23(ASC)]  |PARTITIONED|
                     exchange
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      project ([$$21, $$22])
+                      project ([$$22, $$23])
                       -- 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)
+                          unnest-map [$$21, $$22, $$23] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$20, true, false, false)
                           -- BTREE_SEARCH  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              assign [$$19] <- [150]
+                              assign [$$20] <- [150]
                               -- ASSIGN  |PARTITIONED|
                                 empty-tuple-source
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.adm
index 28c74ac..4e2b676 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.adm
@@ -1,10 +1,10 @@
-distribute result [$$202]
+distribute result [$$210]
 -- DISTRIBUTE_RESULT  |LOCAL|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    aggregate [$$202] <- [agg-sql-sum($$231)]
+    aggregate [$$210] <- [agg-sql-sum($$239)]
     -- AGGREGATE  |LOCAL|
-      aggregate [$$231] <- [agg-sql-count(1)]
+      aggregate [$$239] <- [agg-sql-count(1)]
       -- AGGREGATE  |LOCAL|
         exchange
         -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
@@ -17,14 +17,14 @@
                 project ([])
                 -- STREAM_PROJECT  |PARTITIONED|
                   exchange
-                  -- SORT_MERGE_EXCHANGE [$$134(ASC) ]  |PARTITIONED|
+                  -- SORT_MERGE_EXCHANGE [$$142(ASC) ]  |PARTITIONED|
                     limit 1000
                     -- STREAM_LIMIT  |PARTITIONED|
-                      project ([$$134])
+                      project ([$$142])
                       -- STREAM_PROJECT  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$134, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 10))) limit 1000
+                          data-scan []<-[$$142, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 10))) limit 1000
                           -- DATASOURCE_SCAN  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -37,14 +37,14 @@
                 project ([])
                 -- STREAM_PROJECT  |PARTITIONED|
                   exchange
-                  -- SORT_MERGE_EXCHANGE [$$135(ASC) ]  |PARTITIONED|
+                  -- SORT_MERGE_EXCHANGE [$$143(ASC) ]  |PARTITIONED|
                     limit 1000
                     -- STREAM_LIMIT  |PARTITIONED|
-                      project ([$$135])
+                      project ([$$143])
                       -- STREAM_PROJECT  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$135, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), -10), le($$onek1.getField(2), -1))) limit 1000
+                          data-scan []<-[$$143, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), -10), le($$onek1.getField(2), -1))) limit 1000
                           -- DATASOURCE_SCAN  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |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.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
index db1c3d8..98a3c51 100644
--- 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
@@ -7,12 +7,12 @@
       project ([$$paper])
       -- STREAM_PROJECT  |PARTITIONED|
         exchange
-        -- SORT_MERGE_EXCHANGE [$$14(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$15(ASC) ]  |PARTITIONED|
           limit 10
           -- STREAM_LIMIT  |PARTITIONED|
             exchange
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              data-scan []<-[$$14, $$paper] <- test.DBLP1 condition (contains($$paper.getField(1), "kimL89")) limit 10
+              data-scan []<-[$$15, $$paper] <- test.DBLP1 condition (contains($$paper.getField(1), "kimL89")) limit 10
               -- DATASOURCE_SCAN  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |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.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.adm
index 957c52b..752cbcc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.5.adm
@@ -1,4 +1,4 @@
-distribute result [$$35]
+distribute result [$$37]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
@@ -6,57 +6,57 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        project ([$$35])
+        project ([$$37])
         -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$35] <- [{"dblpid": $$36}]
+          assign [$$37] <- [{"dblpid": $$38}]
           -- ASSIGN  |PARTITIONED|
             limit 2
             -- STREAM_LIMIT  |PARTITIONED|
-              project ([$$36])
+              project ([$$38])
               -- STREAM_PROJECT  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  join (eq($$36, $$39))
-                  -- HYBRID_HASH_JOIN [$$36][$$39]  |PARTITIONED|
+                  join (eq($$38, $$41))
+                  -- HYBRID_HASH_JOIN [$$38][$$41]  |PARTITIONED|
                     exchange
-                    -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                      project ([$$36])
+                    -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                      project ([$$38])
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$36] <- [$$d.getField(1)]
+                        assign [$$38] <- [$$d.getField(1)]
                         -- ASSIGN  |PARTITIONED|
                           project ([$$d])
                           -- STREAM_PROJECT  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$37, $$d] <- test.DBLP1
+                              data-scan []<-[$$39, $$d] <- test.DBLP1
                               -- DATASOURCE_SCAN  |PARTITIONED|
                                 exchange
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   empty-tuple-source
                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                     exchange
-                    -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
-                      project ([$$39])
+                    -- HASH_PARTITION_EXCHANGE [$$41]  |PARTITIONED|
+                      project ([$$41])
                       -- STREAM_PROJECT  |UNPARTITIONED|
-                        assign [$$39] <- [get-item($$28, 0).getField(0).getField(1)]
+                        assign [$$41] <- [get-item($$30, 0).getField(0).getField(1)]
                         -- ASSIGN  |UNPARTITIONED|
-                          aggregate [$$28] <- [listify($$27)]
+                          aggregate [$$30] <- [listify($$29)]
                           -- AGGREGATE  |UNPARTITIONED|
                             limit 1
                             -- STREAM_LIMIT  |UNPARTITIONED|
-                              project ([$$27])
+                              project ([$$29])
                               -- STREAM_PROJECT  |PARTITIONED|
                                 exchange
-                                -- SORT_MERGE_EXCHANGE [$$38(ASC) ]  |PARTITIONED|
-                                  project ([$$38, $$27])
+                                -- SORT_MERGE_EXCHANGE [$$40(ASC) ]  |PARTITIONED|
+                                  project ([$$40, $$29])
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    assign [$$27] <- [{"d": $$d}]
+                                    assign [$$29] <- [{"d": $$d}]
                                     -- ASSIGN  |PARTITIONED|
                                       limit 1
                                       -- STREAM_LIMIT  |PARTITIONED|
                                         exchange
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          data-scan []<-[$$38, $$d] <- test.DBLP1 condition (ends-with($$d.getField(1), "Blakeley95")) limit 1
+                                          data-scan []<-[$$40, $$d] <- test.DBLP1 condition (ends-with($$d.getField(1), "Blakeley95")) limit 1
                                           -- DATASOURCE_SCAN  |PARTITIONED|
                                             exchange
                                             -- ONE_TO_ONE_EXCHANGE  |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.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.adm
index 7509819..b68b6dc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.6.adm
@@ -1,26 +1,26 @@
-distribute result [$$18]
+distribute result [$$19]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 1
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$18])
+      project ([$$19])
       -- STREAM_PROJECT  |PARTITIONED|
         exchange
-        -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
-          project ([$$20, $$18])
+        -- SORT_MERGE_EXCHANGE [$$21(ASC) ]  |PARTITIONED|
+          project ([$$21, $$19])
           -- STREAM_PROJECT  |PARTITIONED|
-            assign [$$18] <- [{"$1": substring($$19, 0, 21)}]
+            assign [$$19] <- [{"$1": substring($$20, 0, 21)}]
             -- ASSIGN  |PARTITIONED|
               limit 1
               -- STREAM_LIMIT  |PARTITIONED|
-                project ([$$20, $$19])
+                project ([$$21, $$20])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$19] <- [$$DBLP1.getField(1)]
+                  assign [$$20] <- [$$DBLP1.getField(1)]
                   -- ASSIGN  |PARTITIONED|
                     exchange
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      data-scan []<-[$$20, $$DBLP1] <- test.DBLP1 condition (gt($$DBLP1.getField(1), "series")) limit 1
+                      data-scan []<-[$$21, $$DBLP1] <- test.DBLP1 condition (gt($$DBLP1.getField(1), "series")) limit 1
                       -- DATASOURCE_SCAN  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |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.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.adm
index 5b408ad..36b851b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.adm
@@ -1,26 +1,26 @@
-distribute result [$$21]
+distribute result [$$22]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 2
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$21])
+      project ([$$22])
       -- STREAM_PROJECT  |PARTITIONED|
         exchange
-        -- SORT_MERGE_EXCHANGE [$$24(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$25(ASC) ]  |PARTITIONED|
           limit 2
           -- STREAM_LIMIT  |PARTITIONED|
-            project ([$$24, $$21])
+            project ([$$25, $$22])
             -- STREAM_PROJECT  |PARTITIONED|
-              assign [$$21] <- [$$25.getField("lang")]
+              assign [$$22] <- [$$26.getField("lang")]
               -- ASSIGN  |PARTITIONED|
-                project ([$$24, $$25])
+                project ([$$25, $$26])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$25] <- [$$t.getField("user")]
+                  assign [$$26] <- [$$t.getField("user")]
                   -- ASSIGN  |PARTITIONED|
                     exchange
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      data-scan []<-[$$24, $$t] <- test.TweetMessages condition (and(ge($$t.getField("user").getField("friends_count"), 0), le($$t.getField("user").getField("friends_count"), 150))) limit 2
+                      data-scan []<-[$$25, $$t] <- test.TweetMessages condition (and(ge($$t.getField("user").getField("friends_count"), 0), le($$t.getField("user").getField("friends_count"), 150))) limit 2
                       -- DATASOURCE_SCAN  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
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
index aaf0c53..e037cf2 100644
--- 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
@@ -7,12 +7,12 @@
       project ([$$paper])
       -- STREAM_PROJECT  |PARTITIONED|
         exchange
-        -- SORT_MERGE_EXCHANGE [$$12(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$13(ASC) ]  |PARTITIONED|
           limit 10
           -- STREAM_LIMIT  |PARTITIONED|
             exchange
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              data-scan []<-[$$12, $$paper] <- test.DBLP1 limit 10
+              data-scan []<-[$$13, $$paper] <- test.DBLP1 limit 10
               -- DATASOURCE_SCAN  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm
index 2176e36..5604580 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm
@@ -7,12 +7,12 @@
       project ([$$paper])
       -- STREAM_PROJECT  |PARTITIONED|
         exchange
-        -- SORT_MERGE_EXCHANGE [$$14(ASC) ]  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$15(ASC) ]  |PARTITIONED|
           limit 10
           -- STREAM_LIMIT  |PARTITIONED|
             exchange
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              data-scan []<-[$$14, $$paper] <- test.DBLP1 limit 10
+              data-scan []<-[$$15, $$paper] <- test.DBLP1 limit 10
               -- DATASOURCE_SCAN  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.adm
index f8a800a..fbe7050 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.adm
@@ -1,10 +1,10 @@
-distribute result [$$180]
+distribute result [$$188]
 -- DISTRIBUTE_RESULT  |LOCAL|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |LOCAL|
-    aggregate [$$180] <- [agg-sql-sum($$205)]
+    aggregate [$$188] <- [agg-sql-sum($$213)]
     -- AGGREGATE  |LOCAL|
-      aggregate [$$205] <- [agg-sql-count(1)]
+      aggregate [$$213] <- [agg-sql-count(1)]
       -- AGGREGATE  |LOCAL|
         exchange
         -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
@@ -17,14 +17,14 @@
                 project ([])
                 -- STREAM_PROJECT  |PARTITIONED|
                   exchange
-                  -- SORT_MERGE_EXCHANGE [$$120(ASC) ]  |PARTITIONED|
+                  -- SORT_MERGE_EXCHANGE [$$128(ASC) ]  |PARTITIONED|
                     limit 100
                     -- STREAM_LIMIT  |PARTITIONED|
-                      project ([$$120])
+                      project ([$$128])
                       -- STREAM_PROJECT  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$120, $$onek1] <- test.onek1 limit 100
+                          data-scan []<-[$$128, $$onek1] <- test.onek1 limit 100
                           -- DATASOURCE_SCAN  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -37,14 +37,14 @@
                 project ([])
                 -- STREAM_PROJECT  |PARTITIONED|
                   exchange
-                  -- SORT_MERGE_EXCHANGE [$$121(ASC) ]  |PARTITIONED|
+                  -- SORT_MERGE_EXCHANGE [$$129(ASC) ]  |PARTITIONED|
                     limit 10
                     -- STREAM_LIMIT  |PARTITIONED|
-                      project ([$$121])
+                      project ([$$129])
                       -- STREAM_PROJECT  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$121, $$onek1] <- test.onek1 limit 10
+                          data-scan []<-[$$129, $$onek1] <- test.onek1 limit 10
                           -- DATASOURCE_SCAN  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.adm
index bb5ac24..513cd19 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.adm
@@ -1,23 +1,23 @@
-distribute result [$$75]
+distribute result [$$80]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     limit 5 offset 5
     -- STREAM_LIMIT  |UNPARTITIONED|
-      project ([$$75])
+      project ([$$80])
       -- STREAM_PROJECT  |PARTITIONED|
-        assign [$$75] <- [get-item($$73, 0)]
+        assign [$$80] <- [get-item($$78, 0)]
         -- ASSIGN  |PARTITIONED|
-          project ([$$73])
+          project ([$$78])
           -- STREAM_PROJECT  |PARTITIONED|
             exchange
-            -- SORT_MERGE_EXCHANGE [$$77(ASC) ]  |PARTITIONED|
-              project ([$$73, $$77])
+            -- SORT_MERGE_EXCHANGE [$$82(ASC) ]  |PARTITIONED|
+              project ([$$78, $$82])
               -- STREAM_PROJECT  |PARTITIONED|
                 subplan {
-                          aggregate [$$73] <- [listify($$72)]
+                          aggregate [$$78] <- [listify($$77)]
                           -- AGGREGATE  |LOCAL|
-                            assign [$$72] <- [object-remove(object-remove(object-remove($$t0, "title"), "authors"), "misc")]
+                            assign [$$77] <- [object-remove(object-remove(object-remove($$t0, "title"), "authors"), "misc")]
                             -- ASSIGN  |LOCAL|
                               unnest $$t0 <- scan-collection(to-array($$paper))
                               -- UNNEST  |LOCAL|
@@ -29,7 +29,7 @@
                   -- STREAM_LIMIT  |PARTITIONED|
                     exchange
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      data-scan []<-[$$77, $$paper] <- test.DBLP1 limit 10
+                      data-scan []<-[$$82, $$paper] <- test.DBLP1 limit 10
                       -- DATASOURCE_SCAN  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/enforcing_item_type/enforcing_item_type.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/enforcing_item_type/enforcing_item_type.1.adm
index f0da628..aec56f6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/enforcing_item_type/enforcing_item_type.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/enforcing_item_type/enforcing_item_type.1.adm
@@ -1,12 +1,12 @@
-distribute result [$$20]
+distribute result [$$21]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$20])
+    project ([$$21])
     -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$20] <- [{"id": get-item($$t, 0), "v": get-item($$t, 1)}]
+      assign [$$21] <- [{"id": get-item($$t, 0), "v": get-item($$t, 1)}]
       -- ASSIGN  |UNPARTITIONED|
-        unnest $$t <- scan-collection(ordered-list-constructor(ordered-list-constructor(29, cast({ f1: "a", f2: 3 }))))
+        unnest $$t <- scan-collection(ordered-list-constructor(ordered-list-constructor(29, cast({ "f1": "a", "f2": 3 }))))
         -- UNNEST  |UNPARTITIONED|
           empty-tuple-source
           -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.1.adm
index 379603f..c757014 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.1.adm
@@ -2,7 +2,7 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    assign [$$5] <- [{ f1: 5, f2: 6, f3: 7 }]
+    assign [$$5] <- [{ "f1": 5, "f2": 6, "f3": 7 }]
     -- ASSIGN  |UNPARTITIONED|
       empty-tuple-source
       -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.adm
index aafd2d0..af11ff1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.adm
@@ -1,8 +1,8 @@
-distribute result [$$14]
+distribute result [$$15]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    assign [$$14] <- [TRUE]
+    assign [$$15] <- [true]
     -- ASSIGN  |UNPARTITIONED|
       project ([])
       -- STREAM_PROJECT  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.5.adm
index 7209f96..c8dd934 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.5.adm
@@ -1,10 +1,10 @@
-distribute result [$$14]
+distribute result [$$15]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$14])
+    project ([$$15])
     -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$14] <- [le($$x, 2)]
+      assign [$$15] <- [le($$x, 2)]
       -- ASSIGN  |UNPARTITIONED|
         exchange
         -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.6.adm
index 351c8e5..e415065 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.6.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.6.adm
@@ -1,10 +1,10 @@
-distribute result [$$14]
+distribute result [$$15]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$14])
+    project ([$$15])
     -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$14] <- [or(null, le($$x, 2))]
+      assign [$$15] <- [or(null, le($$x, 2))]
       -- ASSIGN  |UNPARTITIONED|
         exchange
         -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.adm
index d2f3187..ea7b6ea 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.adm
@@ -1,10 +1,10 @@
-distribute result [$$16]
+distribute result [$$17]
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$16])
+    project ([$$17])
     -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$16] <- [or(TRUE, lt(get-year(current-date()), $$x))]
+      assign [$$17] <- [or(true, lt(get-year(current-date()), $$x))]
       -- ASSIGN  |UNPARTITIONED|
         exchange
         -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_01/metadata_only_01.1.regex b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_01/metadata_only_01.1.regex
index 94ff0a4..7bc4c42 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_01/metadata_only_01.1.regex
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_01/metadata_only_01.1.regex
@@ -1 +1 @@
-/memory\D+917504/
\ No newline at end of file
+/memory\D+917504/
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_01/metadata_only_01.2.regex b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_01/metadata_only_01.2.regex
index 6af6770..acf8405 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_01/metadata_only_01.2.regex
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_01/metadata_only_01.2.regex
@@ -1 +1 @@
-/memory\D+1146880/
\ No newline at end of file
+/memory\D+1146880/
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_02/metadata_only_02.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_02/metadata_only_02.3.adm
new file mode 100644
index 0000000..743bb8a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/metadata_only_02/metadata_only_02.3.adm
@@ -0,0 +1,2 @@
+{ "DataverseName": "test1", "SynonymName": "syn1" }
+{ "DataverseName": "test2", "SynonymName": "syn2" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue453-2/query-issue453-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue453-2/query-issue453-2.1.adm
index 67218de..4b8eef0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue453-2/query-issue453-2.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue453-2/query-issue453-2.1.adm
@@ -1,16 +1,16 @@
-{ "id": 1, "int_m": 1, "int_o": 1, "string_m": "a", "string_o": "a" }
-{ "id": 2, "int_m": 1, "int_o": 1, "string_m": "a", "string_o": null }
-{ "id": 3, "int_m": 1, "int_o": 1, "string_m": "b", "string_o": "a" }
-{ "id": 4, "int_m": 1, "int_o": 1, "string_m": "b", "string_o": null }
-{ "id": 5, "int_m": 1, "int_o": null, "string_m": "a", "string_o": "a" }
-{ "id": 6, "int_m": 1, "int_o": null, "string_m": "a", "string_o": null }
-{ "id": 7, "int_m": 1, "int_o": null, "string_m": "b", "string_o": "a" }
-{ "id": 8, "int_m": 1, "int_o": null, "string_m": "b", "string_o": null }
-{ "id": 9, "int_m": 2, "int_o": 1, "string_m": "a", "string_o": "a" }
-{ "id": 10, "int_m": 2, "int_o": 1, "string_m": "a", "string_o": null }
-{ "id": 11, "int_m": 2, "int_o": 1, "string_m": "b", "string_o": "a" }
-{ "id": 12, "int_m": 2, "int_o": 1, "string_m": "b", "string_o": null }
-{ "id": 13, "int_m": 2, "int_o": null, "string_m": "a", "string_o": "a" }
-{ "id": 14, "int_m": 2, "int_o": null, "string_m": "a", "string_o": null }
-{ "id": 15, "int_m": 2, "int_o": null, "string_m": "b", "string_o": "a" }
-{ "id": 16, "int_m": 2, "int_o": null, "string_m": "b", "string_o": null }
+{ "int_m": 1, "int_o": null, "string_m": "a", "string_o": null }
+{ "int_m": 1, "int_o": null, "string_m": "a", "string_o": "a" }
+{ "int_m": 1, "int_o": null, "string_m": "b", "string_o": null }
+{ "int_m": 1, "int_o": null, "string_m": "b", "string_o": "a" }
+{ "int_m": 1, "int_o": 1, "string_m": "a", "string_o": null }
+{ "int_m": 1, "int_o": 1, "string_m": "a", "string_o": "a" }
+{ "int_m": 1, "int_o": 1, "string_m": "b", "string_o": null }
+{ "int_m": 1, "int_o": 1, "string_m": "b", "string_o": "a" }
+{ "int_m": 2, "int_o": null, "string_m": "a", "string_o": null }
+{ "int_m": 2, "int_o": null, "string_m": "a", "string_o": "a" }
+{ "int_m": 2, "int_o": null, "string_m": "b", "string_o": null }
+{ "int_m": 2, "int_o": null, "string_m": "b", "string_o": "a" }
+{ "int_m": 2, "int_o": 1, "string_m": "a", "string_o": null }
+{ "int_m": 2, "int_o": 1, "string_m": "a", "string_o": "a" }
+{ "int_m": 2, "int_o": 1, "string_m": "b", "string_o": null }
+{ "int_m": 2, "int_o": 1, "string_m": "b", "string_o": "a" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/full-scan/full-scan.3.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/full-scan/full-scan.3.regexjson
index c6f99d1b..0c4fae1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/full-scan/full-scan.3.regexjson
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/full-scan/full-scan.3.regexjson
@@ -1,5 +1,9 @@
 {
     "job-id": "R{[A-Z0-9.:]+}",
+    "create-time": "R{[0-9.]+}",
+    "start-time": "R{[0-9.]+}",
+    "queued-time": "R{.+}",
+    "end-time": "R{[0-9.]+}",
     "counters": [],
     "joblets": [
         {
@@ -29,19 +33,29 @@
                     ],
                     "counters": [
                         {
-                            "name": "Empty Tuple Source",
+                            "name": "R{.+}",
                             "time": "R{[0-9.]+}",
-                            "disk-io": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "Index Search",
-                            "time": "R{[0-9.]+}",
-                            "disk-io": "R{[0-9.]+}"
+                            "runtime-id": "R{.+}"
                         },
                         {
                             "name": "R{.+}",
                             "time": "R{[0-9.]+}",
-                            "disk-io": "R{[0-9.]+}"
+                            "runtime-id": "R{.+}",
+                            "pages-read": "R{[0-9.]+}",
+                            "pages-read-cold": "R{[0-9.]+}",
+                            "cardinality-out": "R{[0-9.]+}",
+                            "avg-tuple-size": "R{[0-9.]+}",
+                            "min-tuple-size": "R{[0-9.]+}",
+                            "max-tuple-size": "R{[0-9.]+}"
+                        },
+                        {
+                            "name": "R{.+}",
+                            "time": "R{[0-9.]+}",
+                            "runtime-id": "R{.+}",
+                            "cardinality-out": "R{[0-9.]+}",
+                            "avg-tuple-size": "R{[0-9.]+}",
+                            "min-tuple-size": "R{[0-9.]+}",
+                            "max-tuple-size": "R{[0-9.]+}"
                         }
                     ]
                 },
@@ -52,18 +66,21 @@
                   "partition-send-profile": [],
                   "counters": [
                     {
-                      "name": "R{.+}",
-                      "time": "R{[0-9.]+}",
-                      "disk-io": "R{[0-9.]+}"
+                        "name": "R{.+}",
+                        "time": "R{[0-9.]+}",
+                        "runtime-id": "R{.+}",
+                        "cardinality-out": "R{[0-9.]+}",
+                        "avg-tuple-size": "R{[0-9.]+}",
+                        "min-tuple-size": "R{[0-9.]+}",
+                        "max-tuple-size": "R{[0-9.]+}"
                     },
                     {
-                      "name": "Result Writer",
-                      "time": "R{[0-9.]+}",
-                      "disk-io": "R{[0-9.]+}"
+                        "name": "R{.+}",
+                        "time": "R{[0-9.]+}",
+                        "runtime-id": "R{.+}"
                     }
                   ]
                 }
             ]
-        }
-    ]
+    }]
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.adm
index e14f391..c69e8a0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.adm
@@ -5,28 +5,28 @@
     project ([$$l])
     -- STREAM_PROJECT  |PARTITIONED|
       exchange
-      -- SORT_MERGE_EXCHANGE [$$16(ASC), $$17(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$17(ASC), $$18(ASC) ]  |PARTITIONED|
         select (eq($$l.getField(10), "1994-01-20"))
         -- STREAM_SELECT  |PARTITIONED|
           exchange
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            unnest-map [$$16, $$17, $$l] <- index-search("LineItem", 0, "tpch", "LineItem", FALSE, FALSE, 2, $$24, $$25, 2, $$24, $$25, TRUE, TRUE, TRUE)
+            unnest-map [$$17, $$18, $$l] <- index-search("LineItem", 0, "tpch", "LineItem", false, false, 2, $$25, $$26, 2, $$25, $$26, true, true, true)
             -- BTREE_SEARCH  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (ASC, $$24) (ASC, $$25)
-                -- STABLE_SORT [$$24(ASC), $$25(ASC)]  |PARTITIONED|
+                order (ASC, $$25) (ASC, $$26)
+                -- STABLE_SORT [$$25(ASC), $$26(ASC)]  |PARTITIONED|
                   exchange
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$24, $$25])
+                    project ([$$25, $$26])
                     -- STREAM_PROJECT  |PARTITIONED|
                       exchange
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        unnest-map [$$23, $$24, $$25] <- index-search("lineitem_shipdateIx", 0, "tpch", "LineItem", FALSE, FALSE, 1, $$21, 1, $$22, TRUE, TRUE, TRUE)
+                        unnest-map [$$24, $$25, $$26] <- index-search("lineitem_shipdateIx", 0, "tpch", "LineItem", false, false, 1, $$22, 1, $$23, true, true, true)
                         -- BTREE_SEARCH  |PARTITIONED|
                           exchange
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            assign [$$21, $$22] <- ["1994-01-20", "1994-01-20"]
+                            assign [$$22, $$23] <- ["1994-01-20", "1994-01-20"]
                             -- ASSIGN  |PARTITIONED|
                               empty-tuple-source
                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.adm
index e14f391..c69e8a0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.adm
@@ -5,28 +5,28 @@
     project ([$$l])
     -- STREAM_PROJECT  |PARTITIONED|
       exchange
-      -- SORT_MERGE_EXCHANGE [$$16(ASC), $$17(ASC) ]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$17(ASC), $$18(ASC) ]  |PARTITIONED|
         select (eq($$l.getField(10), "1994-01-20"))
         -- STREAM_SELECT  |PARTITIONED|
           exchange
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            unnest-map [$$16, $$17, $$l] <- index-search("LineItem", 0, "tpch", "LineItem", FALSE, FALSE, 2, $$24, $$25, 2, $$24, $$25, TRUE, TRUE, TRUE)
+            unnest-map [$$17, $$18, $$l] <- index-search("LineItem", 0, "tpch", "LineItem", false, false, 2, $$25, $$26, 2, $$25, $$26, true, true, true)
             -- BTREE_SEARCH  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (ASC, $$24) (ASC, $$25)
-                -- STABLE_SORT [$$24(ASC), $$25(ASC)]  |PARTITIONED|
+                order (ASC, $$25) (ASC, $$26)
+                -- STABLE_SORT [$$25(ASC), $$26(ASC)]  |PARTITIONED|
                   exchange
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$24, $$25])
+                    project ([$$25, $$26])
                     -- STREAM_PROJECT  |PARTITIONED|
                       exchange
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        unnest-map [$$23, $$24, $$25] <- index-search("lineitem_shipdateIx", 0, "tpch", "LineItem", FALSE, FALSE, 1, $$21, 1, $$22, TRUE, TRUE, TRUE)
+                        unnest-map [$$24, $$25, $$26] <- index-search("lineitem_shipdateIx", 0, "tpch", "LineItem", false, false, 1, $$22, 1, $$23, true, true, true)
                         -- BTREE_SEARCH  |PARTITIONED|
                           exchange
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            assign [$$21, $$22] <- ["1994-01-20", "1994-01-20"]
+                            assign [$$22, $$23] <- ["1994-01-20", "1994-01-20"]
                             -- ASSIGN  |PARTITIONED|
                               empty-tuple-source
                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.10.adm
new file mode 100644
index 0000000..254828e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.10.adm
@@ -0,0 +1,4 @@
+{ "U1": { "user_id": 1, "best_friend": 2, "phones": [ { "kind": "MOBILE", "number": "222-222-2222" } ] }, "U2": { "user_id": 2, "best_friend": 1 } }
+{ "U1": { "user_id": 2, "best_friend": 1 }, "U2": { "user_id": 1, "best_friend": 2, "phones": [ { "kind": "MOBILE", "number": "222-222-2222" } ] } }
+{ "U1": { "user_id": 3, "best_friend": 1, "favorite_color": "Green" }, "U2": { "user_id": 1, "best_friend": 2, "phones": [ { "kind": "MOBILE", "number": "222-222-2222" } ] } }
+{ "U1": { "user_id": 4, "best_friend": null } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.11.adm
new file mode 100644
index 0000000..a177d29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.11.adm
@@ -0,0 +1 @@
+{ "best_friend": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.12.adm
new file mode 100644
index 0000000..eab8cee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.12.adm
@@ -0,0 +1,4 @@
+{ "user_id": 1, "best_friend": 2 }
+{ "user_id": 2, "best_friend": 1 }
+{ "user_id": 3, "best_friend": 1 }
+{ "user_id": 4, "best_friend": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.13.adm
new file mode 100644
index 0000000..62782bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.13.adm
@@ -0,0 +1,3 @@
+{ "A": { "b": 2, "EXCLUDE": 3 } }
+{ "EXCLUDE": 3 }
+{ "EXCLUDE": { "a": 1, "b": 2, "EXCLUDE": 3 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.14.adm
new file mode 100644
index 0000000..d44fb9c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.14.adm
@@ -0,0 +1,2 @@
+{ "user_id": 1, "best_friend": 2, "bestFriends": [ { "user_id": 2, "best_friend": 1, "address": { "zip_code": "99929", "street": "2341 Apple Street" } } ] }
+{ "user_id": 2, "best_friend": 1, "bestFriends": [ { "user_id": 1, "best_friend": 2, "phones": [ { "kind": "MOBILE", "number": "222-222-2222" } ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.3.adm
new file mode 100644
index 0000000..f72a7e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.3.adm
@@ -0,0 +1,4 @@
+{ "user_id": 1, "best_friend": 2 }
+{ "user_id": 2, "best_friend": 1 }
+{ "user_id": 3, "best_friend": 1, "favorite_color": "Green" }
+{ "user_id": 4, "best_friend": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.4.adm
new file mode 100644
index 0000000..ae07f96
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.4.adm
@@ -0,0 +1,4 @@
+{ "user_id": 1, "best_friend": 2, "phones": [ { "kind": "MOBILE", "number": "222-222-2222" } ] }
+{ "user_id": 2, "best_friend": 1 }
+{ "user_id": 3, "best_friend": 1, "favorite_color": "Green" }
+{ "user_id": 4, "best_friend": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.5.adm
new file mode 100644
index 0000000..16d3516
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.5.adm
@@ -0,0 +1,2 @@
+[ { "user_id": 2, "best_friend": 1 } ]
+[ { "user_id": 3, "best_friend": 1, "favorite_color": "Green" } ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.6.adm
new file mode 100644
index 0000000..bf58f6a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.6.adm
@@ -0,0 +1,4 @@
+{ "user_id": 1 }
+{ "user_id": 2 }
+{ "user_id": 3 }
+{ "user_id": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.7.adm
new file mode 100644
index 0000000..71d7a13
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.7.adm
@@ -0,0 +1,3 @@
+{ "best_friend": 1 }
+{ "best_friend": 2 }
+{ "best_friend": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.8.adm
new file mode 100644
index 0000000..3c5ba18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.8.adm
@@ -0,0 +1 @@
+{ "user_id": 1, "best_friend": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.9.adm
new file mode 100644
index 0000000..c8cbba5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/complex-exclude/complex-exclude.9.adm
@@ -0,0 +1,4 @@
+{ "U": { "user_id": 1, "best_friend": 2 }, "miscInfo": { "24as": "23412" } }
+{ "U": { "user_id": 2, "best_friend": 1 }, "miscInfo": { "24as": "23412" } }
+{ "U": { "user_id": 3, "best_friend": 1, "favorite_color": "Green" }, "miscInfo": { "24as": "23412" } }
+{ "U": { "user_id": 4, "best_friend": null }, "miscInfo": { "24as": "23412" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.3.adm
new file mode 100644
index 0000000..c27da48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.3.adm
@@ -0,0 +1,12 @@
+{ "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
+{ "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
+{ "user": { "screen-name": "NilaMilliron_tw", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }
+{ "user": { "screen-name": "OliJackson_512", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }
+{ "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }
+{ "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }
+{ "user": { "screen-name": "ChangEwing_573", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
+{ "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "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/select-exclude/tiny-social/tiny-social.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.4.adm
new file mode 100644
index 0000000..0b3ee31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.4.adm
@@ -0,0 +1,11 @@
+{ "TM": { "tweetid": "1", "user": { "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.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" } }
+{ "TM": { "tweetid": "10", "user": { "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" } }
+{ "TM": { "tweetid": "11", "user": { "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" } }
+{ "TM": { "tweetid": "2", "user": { "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" } }
+{ "TM": { "tweetid": "3", "user": { "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" } }
+{ "TM": { "tweetid": "4", "user": { "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" } }
+{ "TM": { "tweetid": "5", "user": { "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" } }
+{ "TM": { "tweetid": "6", "user": { "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" } }
+{ "TM": { "tweetid": "7", "user": { "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" } }
+{ "TM": { "tweetid": "8", "user": { "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" } }
+{ "TM": { "tweetid": "9", "user": { "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000"), "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/select-exclude/tiny-social/tiny-social.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.5.adm
new file mode 100644
index 0000000..0621911
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.5.adm
@@ -0,0 +1,24 @@
+{ "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
+{ "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)", "send-time": datetime("2008-04-26T10:10:00.000") }
+{ "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
+{ "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(", "send-time": datetime("2008-01-26T10:10:00.000") }
+{ "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }
+{ "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible", "send-time": datetime("2008-03-09T10:10:00.000") }
+{ "tweetid": "12", "user": { "screen-name": "OliJackson_512", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }
+{ "tweetid": "12", "user": { "screen-name": "OliJackson_512", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)", "send-time": datetime("2010-02-13T10:10:00.000") }
+{ "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
+{ "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)", "send-time": datetime("2010-05-13T10:10:00.000") }
+{ "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
+{ "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)", "send-time": datetime("2006-11-04T10:10:00.000") }
+{ "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
+{ "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)", "send-time": datetime("2011-12-26T10:10:00.000") }
+{ "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }
+{ "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(", "send-time": datetime("2006-08-04T10:10:00.000") }
+{ "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }
+{ "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)", "send-time": datetime("2010-05-07T10:10:00.000") }
+{ "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
+{ "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good", "send-time": datetime("2011-08-25T10:10:00.000") }
+{ "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }
+{ "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)", "send-time": datetime("2005-10-14T10:10:00.000") }
+{ "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
+{ "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome", "send-time": datetime("2012-07-21T10:10:00.000") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.6.adm
new file mode 100644
index 0000000..81592e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/select-exclude/tiny-social/tiny-social.6.adm
@@ -0,0 +1,12 @@
+{ "TM": { "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" } }
+{ "TM": { "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" } }
+{ "TM": { "user": { "screen-name": "NilaMilliron_tw", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" } }
+{ "TM": { "user": { "screen-name": "OliJackson_512", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" } }
+{ "TM": { "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" } }
+{ "TM": { "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" } }
+{ "TM": { "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" } }
+{ "TM": { "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" } }
+{ "TM": { "user": { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" } }
+{ "TM": { "user": { "screen-name": "ChangEwing_573", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" } }
+{ "TM": { "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" } }
+{ "TM": { "user": { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "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/tpcds/q18/q18.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpcds/q18/q18.2.adm
new file mode 100644
index 0000000..6280446
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpcds/q18/q18.2.adm
@@ -0,0 +1 @@
+{ "cnt": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpcds/q18/q18.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpcds/q18/q18.3.adm
new file mode 100644
index 0000000..6280446
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpcds/q18/q18.3.adm
@@ -0,0 +1 @@
+{ "cnt": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.11.adm
index a809a8e..eb1177f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.11.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.11.adm
@@ -6,29 +6,29 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        union ($$151, $$178, $$t)
+        union ($$161, $$188, $$t)
         -- UNION_ALL  |PARTITIONED|
           exchange
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            union ($$213, $$227, $$151)
+            union ($$223, $$237, $$161)
             -- UNION_ALL  |PARTITIONED|
               exchange
               -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                project ([$$213])
+                project ([$$223])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$213] <- [{"two": $$183}]
+                  assign [$$223] <- [{"two": $$193}]
                   -- ASSIGN  |PARTITIONED|
                     limit 4
                     -- STREAM_LIMIT  |PARTITIONED|
-                      project ([$$183])
+                      project ([$$193])
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$183] <- [$$onek1.getField(2)]
+                        assign [$$193] <- [$$onek1.getField(2)]
                         -- ASSIGN  |PARTITIONED|
                           project ([$$onek1])
                           -- STREAM_PROJECT  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$187, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 10))) limit 4
+                              data-scan []<-[$$197, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 10))) limit 4
                               -- DATASOURCE_SCAN  |PARTITIONED|
                                 exchange
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -36,21 +36,21 @@
                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
               exchange
               -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                project ([$$227])
+                project ([$$237])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$227] <- [{"two": $$184}]
+                  assign [$$237] <- [{"two": $$194}]
                   -- ASSIGN  |PARTITIONED|
                     limit 4
                     -- STREAM_LIMIT  |PARTITIONED|
-                      project ([$$184])
+                      project ([$$194])
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$184] <- [$$onek2.getField(2)]
+                        assign [$$194] <- [$$onek2.getField(2)]
                         -- ASSIGN  |PARTITIONED|
                           project ([$$onek2])
                           -- STREAM_PROJECT  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$188, $$onek2] <- test.onek2 condition (and(ge($$onek2.getField(2), 1), le($$onek2.getField(2), 100))) limit 4
+                              data-scan []<-[$$198, $$onek2] <- test.onek2 condition (and(ge($$onek2.getField(2), 1), le($$onek2.getField(2), 100))) limit 4
                               -- DATASOURCE_SCAN  |PARTITIONED|
                                 exchange
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -58,25 +58,25 @@
                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
           exchange
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            union ($$345, $$354, $$178)
+            union ($$355, $$364, $$188)
             -- UNION_ALL  |PARTITIONED|
               exchange
               -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                project ([$$345])
+                project ([$$355])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$345] <- [{"two": $$185}]
+                  assign [$$355] <- [{"two": $$195}]
                   -- ASSIGN  |PARTITIONED|
                     limit 4
                     -- STREAM_LIMIT  |PARTITIONED|
-                      project ([$$185])
+                      project ([$$195])
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$185] <- [$$onek1.getField(2)]
+                        assign [$$195] <- [$$onek1.getField(2)]
                         -- ASSIGN  |PARTITIONED|
                           project ([$$onek1])
                           -- STREAM_PROJECT  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$189, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 1000))) limit 4
+                              data-scan []<-[$$199, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 1000))) limit 4
                               -- DATASOURCE_SCAN  |PARTITIONED|
                                 exchange
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -84,21 +84,21 @@
                                   -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
               exchange
               -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
-                project ([$$354])
+                project ([$$364])
                 -- STREAM_PROJECT  |PARTITIONED|
-                  assign [$$354] <- [{"two": $$186}]
+                  assign [$$364] <- [{"two": $$196}]
                   -- ASSIGN  |PARTITIONED|
                     limit 4
                     -- STREAM_LIMIT  |PARTITIONED|
-                      project ([$$186])
+                      project ([$$196])
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$186] <- [$$onek2.getField(2)]
+                        assign [$$196] <- [$$onek2.getField(2)]
                         -- ASSIGN  |PARTITIONED|
                           project ([$$onek2])
                           -- STREAM_PROJECT  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              data-scan []<-[$$190, $$onek2] <- test.onek2 condition (and(ge($$onek2.getField(2), 1), le($$onek2.getField(2), 10000))) limit 4
+                              data-scan []<-[$$200, $$onek2] <- test.onek2 condition (and(ge($$onek2.getField(2), 1), le($$onek2.getField(2), 10000))) limit 4
                               -- DATASOURCE_SCAN  |PARTITIONED|
                                 exchange
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.9.adm
index 4a46e2d..d4c025f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.9.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_opt_1/union_opt_1.9.adm
@@ -6,25 +6,25 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       exchange
       -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-        union ($$48, $$54, $$t)
+        union ($$52, $$58, $$t)
         -- UNION_ALL  |PARTITIONED|
           exchange
           -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
             limit 4
             -- STREAM_LIMIT  |PARTITIONED|
-              project ([$$48])
+              project ([$$52])
               -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$48] <- [{"two": $$103}]
+                assign [$$52] <- [{"two": $$107}]
                 -- ASSIGN  |PARTITIONED|
-                  project ([$$103])
+                  project ([$$107])
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$103] <- [$$onek1.getField(2)]
+                    assign [$$107] <- [$$onek1.getField(2)]
                     -- ASSIGN  |PARTITIONED|
                       project ([$$onek1])
                       -- STREAM_PROJECT  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$61, $$onek1] <- test.onek1 condition (gt($$onek1.getField(2), 0)) limit 4
+                          data-scan []<-[$$65, $$onek1] <- test.onek1 condition (gt($$onek1.getField(2), 0)) limit 4
                           -- DATASOURCE_SCAN  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -34,19 +34,19 @@
           -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
             limit 4
             -- STREAM_LIMIT  |PARTITIONED|
-              project ([$$54])
+              project ([$$58])
               -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$54] <- [{"two": $$105}]
+                assign [$$58] <- [{"two": $$109}]
                 -- ASSIGN  |PARTITIONED|
-                  project ([$$105])
+                  project ([$$109])
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$105] <- [$$onek2.getField(2)]
+                    assign [$$109] <- [$$onek2.getField(2)]
                     -- ASSIGN  |PARTITIONED|
                       project ([$$onek2])
                       -- STREAM_PROJECT  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$62, $$onek2] <- test.onek2 condition (gt($$onek2.getField(2), 0)) limit 4
+                          data-scan []<-[$$66, $$onek2] <- test.onek2 condition (gt($$onek2.getField(2), 0)) limit 4
                           -- DATASOURCE_SCAN  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.03.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.03.regexadm
index 519019a..7029ce2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.03.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.03.regexadm
@@ -31,6 +31,7 @@
 \s*\Q"metrics": {\E
 \s*\Q"elapsedTime": "\E[^"]+\Q",\E
 \s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
 \s*\Q"resultCount": \E[0-9]+\Q,\E
 \s*\Q"resultSize": \E[0-9]+\Q,\E
 \s*\Q"processedObjects": \E[0-9]+\Q,\E
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.04.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.04.regexadm
index 8eb28ea..8478d17 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.04.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.04.regexadm
@@ -24,6 +24,7 @@
 \s*\Q"metrics": {\E
 \s*\Q"elapsedTime": "\E[^"]+\Q",\E
 \s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
 \s*\Q"resultCount": \E[0-9]+\Q,\E
 \s*\Q"resultSize": \E[0-9]+\Q,\E
 \s*\Q"processedObjects": \E[0-9]+\Q,\E
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.05.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.05.regexadm
index 5c477e7..5399ed8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.05.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.05.regexadm
@@ -24,6 +24,7 @@
 \s*\Q"metrics": {\E
 \s*\Q"elapsedTime": "\E[^"]+\Q",\E
 \s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
 \s*\Q"resultCount": \E[0-9]+\Q,\E
 \s*\Q"resultSize": \E[0-9]+\Q,\E
 \s*\Q"processedObjects": \E[0-9]+\Q,\E
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.06.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.06.regexadm
index 5eeb3df..aa58fb4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.06.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.06.regexadm
@@ -12,6 +12,7 @@
 \s*\Q"metrics": {\E
 \s*\Q"elapsedTime": "\E[^"]+\Q",\E
 \s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
 \s*\Q"resultCount": \E[0-9]+\Q,\E
 \s*\Q"resultSize": \E[0-9]+\Q,\E
 \s*\Q"processedObjects": \E[0-9]+\Q,\E
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.07.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.07.regexadm
index 5c477e7..5399ed8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.07.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.07.regexadm
@@ -24,6 +24,7 @@
 \s*\Q"metrics": {\E
 \s*\Q"elapsedTime": "\E[^"]+\Q",\E
 \s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
 \s*\Q"resultCount": \E[0-9]+\Q,\E
 \s*\Q"resultSize": \E[0-9]+\Q,\E
 \s*\Q"processedObjects": \E[0-9]+\Q,\E
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.08.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.08.regexadm
index 519019a..7029ce2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.08.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/warnings/warnings-limit/warnings-limit.08.regexadm
@@ -31,6 +31,7 @@
 \s*\Q"metrics": {\E
 \s*\Q"elapsedTime": "\E[^"]+\Q",\E
 \s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
 \s*\Q"resultCount": \E[0-9]+\Q,\E
 \s*\Q"resultSize": \E[0-9]+\Q,\E
 \s*\Q"processedObjects": \E[0-9]+\Q,\E
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/open-closed/query-issue453-2/query-issue453-2.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/open-closed/query-issue453-2/query-issue453-2.3.ast
index ffedb2d..057e63a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/open-closed/query-issue453-2/query-issue453-2.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/open-closed/query-issue453-2/query-issue453-2.3.ast
@@ -1,7 +1,26 @@
 DataverseUse test
 Query:
-SELECT ELEMENT [
-Variable [ Name=$d ]
+SELECT [
+FieldAccessor [
+  Variable [ Name=$d ]
+  Field=int_m
+]
+int_m
+FieldAccessor [
+  Variable [ Name=$d ]
+  Field=int_o
+]
+int_o
+FieldAccessor [
+  Variable [ Name=$d ]
+  Field=string_m
+]
+string_m
+FieldAccessor [
+  Variable [ Name=$d ]
+  Field=string_o
+]
+string_o
 ]
 FROM [  FunctionCall asterix.dataset@1[
     LiteralExpr [STRING] [test.DataOpen]
@@ -11,7 +30,22 @@
 Orderby
   FieldAccessor [
     Variable [ Name=$d ]
-    Field=id
+    Field=int_m
+  ]
+  ASC
+  FieldAccessor [
+    Variable [ Name=$d ]
+    Field=int_o
+  ]
+  ASC
+  FieldAccessor [
+    Variable [ Name=$d ]
+    Field=string_m
+  ]
+  ASC
+  FieldAccessor [
+    Variable [ Name=$d ]
+    Field=string_o
   ]
   ASC
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml
index 686ede2..284c2fd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml
@@ -52,15 +52,7 @@
     <test-case FilePath="external-library" check-warnings="true">
       <compilation-unit name="py_function_error">
         <output-dir compare="Clean-JSON">py_function_error</output-dir>
-        <expected-warn>ASX0201: External UDF returned exception. Returned exception was: Traceback (most recent call last):
-  File "entrypoint.py", line 181, in handle_call
-    result[0].append(self.next_tuple(*arg, key=self.mid))
-  File "entrypoint.py", line 99, in next_tuple
-    return self.wrapped_fns[key](*args)
-  File "site-packages/roundtrip.py", line 32, in warning
-    raise ArithmeticError("oof")
-ArithmeticError: oof
- (in line 28, at column 1)</expected-warn>
+        <expected-warn>ArithmeticError: oof</expected-warn>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-library">
@@ -76,8 +68,8 @@
     <test-case FilePath="external-library" check-warnings="true">
       <compilation-unit name="crash">
         <output-dir compare="Text">crash</output-dir>
-        <expected-warn>ASX0201: External UDF returned exception. Returned exception was: Function externallibtest:crash#0 failed to execute (in line 23, at column 1)</expected-warn>
-        <expected-warn>ASX0201: External UDF returned exception. Returned exception was: java.io.IOException: Python process exited with code: 1 (in line 23, at column 1)</expected-warn>
+        <expected-warn>ASX0201: External UDF returned exception.</expected-warn>
+        <expected-warn>ASX0201: External UDF returned exception.</expected-warn>
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-library">
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 c0784c1..5a544b2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -34,6 +34,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="api">
+      <compilation-unit name="compileonly-2">
+        <output-dir compare="Text">compileonly-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
       <compilation-unit name="multiple-param-values">
         <output-dir compare="Text">multiple-param-values</output-dir>
       </compilation-unit>
@@ -3603,10 +3608,10 @@
     <test-case FilePath="comparison" check-warnings="true">
       <compilation-unit name="incomparable_types">
         <output-dir compare="Text">incomparable_types</output-dir>
-          <expected-warn>Incomparable input types: string and bigint (in line 25, at column 13)</expected-warn>
+          <expected-warn>Incomparable input types: string and bigint (in line 26, at column 13)</expected-warn>
           <expected-warn>Incomparable input types: array and bigint (in line 23, at column 7)</expected-warn>
           <expected-warn>Incomparable input types: point and point (in line 24, at column 18)</expected-warn>
-          <expected-warn>Incomparable input types: bigint and string (in line 24, at column 46)</expected-warn>
+          <expected-warn>Incomparable input types: bigint and string (in line 25, at column 46)</expected-warn>
       </compilation-unit>
     </test-case>
   </test-group>
@@ -4120,6 +4125,11 @@
         -->
   </test-group>
   <test-group name="ddl">
+    <test-case FilePath="ddl">
+      <compilation-unit name="analyze-dataset-1">
+        <output-dir compare="Text">analyze-dataset-1</output-dir>
+      </compilation-unit>
+    </test-case>
     <test-case FilePath="ddl/create-index">
       <compilation-unit name="create-index-1">
         <output-dir compare="Text">create-index-1</output-dir>
@@ -6656,6 +6666,16 @@
         <output-dir compare="Text">join-with-empty-dataset</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="join">
+      <compilation-unit name="nlj_partitioning_property">
+        <output-dir compare="Text">nlj_partitioning_property</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="join">
+      <compilation-unit name="inner_right_corr">
+        <output-dir compare="Text">inner_right_corr</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="list">
     <test-case FilePath="list">
@@ -9904,6 +9924,25 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="select-exclude">
+    <test-case FilePath="select-exclude">
+      <compilation-unit name="tiny-social">
+        <output-dir compare="Text">tiny-social</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="select-exclude">
+      <compilation-unit name="complex-exclude">
+        <output-dir compare="Text">complex-exclude</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="select-exclude">
+      <compilation-unit name="exclude-negative">
+        <output-dir compare="Text">exclude-negative</output-dir>
+        <expected-error>ASX1001: Syntax error</expected-error>
+        <expected-error>ASX1001: Syntax error</expected-error>
+      </compilation-unit>
+    </test-case>
+  </test-group>
   <test-group name="semistructured">
     <test-case FilePath="semistructured">
       <compilation-unit name="count-nullable">
@@ -15371,9 +15410,9 @@
         <output-dir compare="Text">inapplicable-hint-warning</output-dir>
         <expected-warn>HYR10006: Could not apply Group By hint: hash</expected-warn>
         <expected-warn>ASX1107: Unexpected hint: indexnl. "hash" expected at this location</expected-warn>
-        <expected-warn>ASX1107: Unexpected hint: hash. "hash-bcast", "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
-        <expected-warn>ASX1107: Unexpected hint: auto. "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
-        <expected-warn>ASX1107: Unexpected hint: hash. "indexnl", "range", "skip-index", "spatial-partitioning", "use-index" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: hash. "hash-bcast", "indexnl", "hashjoin", "skip-index", "use-index", "selectivity", "productivity" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: auto. "indexnl", "skip-index", "hashjoin", "use-index", "selectivity" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: hash. "indexnl", "range", "hashjoin", "skip-index", "spatial-partitioning", "use-index" expected at this location</expected-warn>
         <expected-warn>ASX1107: Unexpected hint: hash. None expected at this location</expected-warn>
       </compilation-unit>
     </test-case>
@@ -15394,12 +15433,12 @@
       <compilation-unit name="unknown-hint-warning">
         <output-dir compare="Text">unknown-hint-warning</output-dir>
         <expected-warn>ASX1107: Unexpected hint: unknown_hint_groupby. "hash" expected at this location</expected-warn>
-        <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr. "hash-bcast", "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
-        <expected-warn>ASX1107: Unexpected hint: unknown_hint_between. "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
-        <expected-warn>ASX1107: Unexpected hint: unknown_hint_funcall. "indexnl", "range", "skip-index", "spatial-partitioning", "use-index" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr. "hash-bcast", "indexnl", "hashjoin", "skip-index", "use-index", "selectivity", "productivity" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: unknown_hint_between. "indexnl", "skip-index", "hashjoin", "use-index", "selectivity" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: unknown_hint_funcall. "indexnl", "range", "hashjoin", "skip-index", "spatial-partitioning", "use-index" expected at this location</expected-warn>
         <expected-warn>ASX1107: Unexpected hint: unknown_hint_elsewhere. None expected at this location</expected-warn>
-        <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr_6. "hash-bcast", "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
-        <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr_6. "hash-bcast", "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr_6. "hash-bcast", "indexnl", "hashjoin", "skip-index", "use-index", "selectivity", "productivity" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr_6. "hash-bcast", "indexnl", "hashjoin", "skip-index", "use-index", "selectivity", "productivity" expected at this location</expected-warn>
       </compilation-unit>
     </test-case>
     <test-case FilePath="warnings">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_batch_lookup.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_batch_lookup.xml
new file mode 100644
index 0000000..fd8b886
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_batch_lookup.xml
@@ -0,0 +1,862 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT 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-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+  <test-group name="batched-lookups">
+    <test-case FilePath="dml">
+      <compilation-unit name="compact-dataset-and-its-indexes">
+        <output-dir compare="Text">compact-dataset-and-its-indexes</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-constant-merge-policy">
+        <output-dir compare="Text">using-constant-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-prefix-merge-policy">
+        <output-dir compare="Text">using-prefix-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-concurrent-merge-policy">
+        <output-dir compare="Text">using-concurrent-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-correlated-prefix-merge-policy">
+        <output-dir compare="Text">using-correlated-prefix-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-correlated-prefix-merge-policy-with-feed">
+        <output-dir compare="Text">using-correlated-prefix-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-no-merge-policy">
+        <output-dir compare="Text">using-no-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="delete-from-loaded-dataset-with-index">
+        <output-dir compare="Text">delete-from-loaded-dataset-with-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="load-with-index">
+        <output-dir compare="Text">load-with-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-btree-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-btree-correlated-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-btree-correlated-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-btree-secondary-index-open">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-btree-correlated-secondary-index-open">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-rtree-secondary-index-open">
+        <output-dir compare="Text">scan-delete-rtree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-rtree-correlated-secondary-index-open">
+        <output-dir compare="Text">scan-delete-rtree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-btree-correlated-secondary-index-open">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-rtree-correlated-secondary-index-open">
+        <output-dir compare="Text">scan-insert-rtree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-index-01">
+        <output-dir compare="Text">fulltext-index-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-index-02">
+        <output-dir compare="Text">fulltext-index-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-index-08">
+        <output-dir compare="Text">fulltext-index-08</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-index-09">
+        <output-dir compare="Text">fulltext-index-09</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="stopwords-full-text-filter-1">
+        <output-dir compare="Text">stopwords-full-text-filter-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-index-nested-loop-join">
+        <output-dir compare="Text">btree-index-nested-loop-join</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-pidx-to-sidx-idxonly-equi-join_01">
+        <output-dir compare="Text">btree-pidx-to-sidx-idxonly-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-secondary-equi-join_02">
+        <output-dir compare="Text">btree-secondary-equi-join_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-secondary-equi-join_03">
+        <output-dir compare="Text">btree-secondary-equi-join_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-secondary-equi-join_04">
+        <output-dir compare="Text">btree-secondary-equi-join_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-sidx-idxonly-to-pidx-equi-join_01">
+        <output-dir compare="Text">btree-sidx-idxonly-to-pidx-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-sidx-idxonly-to-sidx-idxonly-equi-join_01">
+        <output-dir compare="Text">btree-sidx-idxonly-to-sidx-idxonly-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-sidx-non-idxonly-to-pidx-equi-join_01">
+        <output-dir compare="Text">btree-sidx-non-idxonly-to-pidx-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-sidx-non-idxonly-to-sidx-idxonly-equi-join_01">
+        <output-dir compare="Text">btree-sidx-non-idxonly-to-sidx-idxonly-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join" check-warnings="true">
+      <compilation-unit name="hints-indexnl-params">
+        <output-dir compare="Text">hints-indexnl-params</output-dir>
+        <expected-warn>ASX1132: Invalid specification for hint indexnl. ASX1001: Syntax error: In line 1 &gt;&gt;(8, idx_tenk2_1k_2k)&lt;&lt; Encountered &lt;INTEGER_LITERAL&gt; "8" at column 2.  (in line 35, at column 21)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="rtree-spatial-intersect-point_05">
+        <output-dir compare="Text">rtree-spatial-intersect-point_05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-index-composite-key-03">
+        <output-dir compare="Text">btree-index-composite-key-03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-index-composite-key-mixed-intervals">
+        <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-index-rewrite-multiple">
+        <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sidx-composite-idxonly-01">
+        <output-dir compare="Text">btree-sidx-composite-idxonly-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sidx-composite-idxonly-02">
+        <output-dir compare="Text">btree-sidx-composite-idxonly-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sidx-composite-idxonly-03">
+        <output-dir compare="Text">btree-sidx-composite-idxonly-03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sidx-composite-idxonly-04">
+        <output-dir compare="Text">btree-sidx-composite-idxonly-04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sidx-idxonly-01">
+        <output-dir compare="Text">btree-sidx-idxonly-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="cust-index-age-nullable">
+        <output-dir compare="Text">cust-index-age-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="intersection-with-between">
+        <output-dir compare="Text">intersection-with-between</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-ngram-edit-distance-large-data">
+        <output-dir compare="Text">inverted-index-ngram-edit-distance-large-data</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-olist-edit-distance">
+        <output-dir compare="Text">inverted-index-olist-edit-distance</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-olist-jaccard">
+        <output-dir compare="Text">inverted-index-olist-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-ulist-jaccard">
+        <output-dir compare="Text">inverted-index-ulist-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="orders-index-custkey-conjunctive-open">
+        <output-dir compare="Text">orders-index-custkey-conjunctive-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="orders-index-custkey-conjunctive">
+        <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="orders-index-custkey-open">
+        <output-dir compare="Text">orders-index-custkey-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="orders-index-custkey">
+        <output-dir compare="Text">orders-index-custkey</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="range-search-open">
+        <output-dir compare="Text">range-search-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="range-search">
+        <output-dir compare="Text">range-search</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="rtree-sidx-non-idxonly-02">
+        <output-dir compare="Text">rtree-sidx-non-idxonly-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="dataset-with-meta">
+        <output-dir compare="Text">dataset-with-meta</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection" check-warnings="true">
+      <compilation-unit name="hints-skip-index">
+        <output-dir compare="Text">hints-skip-index</output-dir>
+        <expected-warn>ASX1132: Invalid specification for hint skip-index. ASX1001: Syntax error: In line 1 &gt;&gt;(13, idx_1k)&lt;&lt; Encountered &lt;INTEGER_LITERAL&gt; "13" at column 2.  (in line 32, at column 19)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection" check-warnings="true">
+      <compilation-unit name="hints-use-index">
+        <output-dir compare="Text">hints-use-index</output-dir>
+        <expected-warn>ASX1132: Invalid specification for hint use-index. ASX1001: Syntax error: In line 1 &gt;&gt;(18, idx_1k_2k)&lt;&lt; Encountered &lt;INTEGER_LITERAL&gt; "18" at column 2.  (in line 33, at column 15)</expected-warn>
+        <expected-warn>ASX1132: Invalid specification for hint use-index. ASX1001: Syntax error: In line 1 &gt;&gt;()&lt;&lt; Encountered ")" at column 2.  (in line 33, at column 15)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="verify">
+        <output-dir compare="Text">verify</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-enforced/index-join">
+      <compilation-unit name="ngram-jaccard">
+        <output-dir compare="Text">ngram-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-enforced/index-join">
+      <compilation-unit name="ngram-jaccard-inline">
+        <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-enforced/index-selection">
+      <compilation-unit name="btree-index-composite-key-mixed-intervals">
+        <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-enforced/index-selection">
+      <compilation-unit name="btree-index-rewrite-multiple">
+        <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-enforced/index-selection">
+      <compilation-unit name="orders-index-custkey">
+        <output-dir compare="Text">orders-index-custkey</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-enforced/index-selection">
+      <compilation-unit name="orders-index-custkey-conjunctive">
+        <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-enforced/index-selection">
+      <compilation-unit name="range-search">
+        <output-dir compare="Text">range-search</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-non-enforced/index-selection">
+      <compilation-unit name="btree-index-01">
+        <output-dir compare="Text">btree-index-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-non-enforced/index-selection">
+      <compilation-unit name="btree-index-02">
+        <output-dir compare="Text">btree-index-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-non-enforced/index-selection">
+      <compilation-unit name="btree-index-04">
+        <output-dir compare="Text">btree-index-04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-non-enforced/correlated-index-selection">
+      <compilation-unit name="btree-index-01">
+        <output-dir compare="Text">btree-index-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-index-non-enforced/index-join">
+      <compilation-unit name="btree-equi-join-01">
+        <output-dir compare="Text">btree-equi-join-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-join">
+      <compilation-unit name="ngram-jaccard">
+        <output-dir compare="Text">ngram-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-join">
+      <compilation-unit name="ngram-jaccard-inline">
+        <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-join">
+      <compilation-unit name="btree-secondary-non-enforced-equi-join">
+        <output-dir compare="Text">btree-secondary-non-enforced-equi-join</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-join">
+      <compilation-unit name="btree-secondary-non-enforced-equi-join-2">
+        <output-dir compare="Text">btree-secondary-non-enforced-equi-join-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-selection">
+      <compilation-unit name="btree-index-composite-key-mixed-intervals">
+        <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-selection">
+      <compilation-unit name="btree-index-rewrite-multiple">
+        <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-selection">
+      <compilation-unit name="orders-index-custkey">
+        <output-dir compare="Text">orders-index-custkey</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-selection">
+      <compilation-unit name="orders-index-custkey-conjunctive">
+        <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-selection">
+      <compilation-unit name="range-search">
+        <output-dir compare="Text">range-search</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-selection">
+      <compilation-unit name="non-enforced-01">
+        <output-dir compare="Text">non-enforced-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-selection">
+      <compilation-unit name="non-enforced-02">
+        <output-dir compare="Text">non-enforced-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-open-index/index-selection">
+      <compilation-unit name="non-enforced-04">
+        <output-dir compare="Text">non-enforced-04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="btree-index-composite-key-mixed-intervals">
+        <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="btree-index-rewrite-multiple">
+        <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="cust-index-age-nullable">
+        <output-dir compare="Text">cust-index-age-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="inverted-index-olist-edit-distance">
+        <output-dir compare="Text">inverted-index-olist-edit-distance</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="inverted-index-olist-jaccard">
+        <output-dir compare="Text">inverted-index-olist-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="inverted-index-ulist-jaccard">
+        <output-dir compare="Text">inverted-index-ulist-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="orders-index-custkey">
+        <output-dir compare="Text">orders-index-custkey</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="orders-index-custkey-conjunctive">
+        <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="orders-index-custkey-conjunctive-open">
+        <output-dir compare="Text">orders-index-custkey-conjunctive-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="orders-index-custkey-open">
+        <output-dir compare="Text">orders-index-custkey-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="range-search">
+        <output-dir compare="Text">range-search</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index/index-selection">
+      <compilation-unit name="range-search-open">
+        <output-dir compare="Text">range-search-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-case FilePath="nested-index-dml">
+      <compilation-unit name="compact-dataset-and-its-indexes">
+        <output-dir compare="Text">compact-dataset-and-its-indexes</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="delete-from-loaded-dataset-with-index">
+        <output-dir compare="Text">delete-from-loaded-dataset-with-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="insert-into-loaded-dataset-with-index_01">
+        <output-dir compare="Text">insert-into-loaded-dataset-with-index_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="insert-into-loaded-dataset-with-index_02">
+        <output-dir compare="Text">insert-into-loaded-dataset-with-index_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="load-with-ngram-index">
+        <output-dir compare="Text">load-with-ngram-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="load-with-rtree-index">
+        <output-dir compare="Text">load-with-rtree-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="load-with-word-index">
+        <output-dir compare="Text">load-with-word-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-delete-btree-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-delete-rtree-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-rtree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-delete-rtree-secondary-index">
+        <output-dir compare="Text">scan-delete-rtree-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/closed">
+      <compilation-unit name="use-case-1">
+        <output-dir compare="Text">use-case-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/open">
+      <compilation-unit name="use-case-1">
+        <output-dir compare="Text">use-case-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/closed">
+      <compilation-unit name="use-case-2">
+        <output-dir compare="Text">use-case-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/open">
+      <compilation-unit name="use-case-2">
+        <output-dir compare="Text">use-case-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/closed">
+      <compilation-unit name="use-case-3">
+        <output-dir compare="Text">use-case-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/open">
+      <compilation-unit name="use-case-3">
+        <output-dir compare="Text">use-case-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/closed">
+      <compilation-unit name="use-case-4">
+        <output-dir compare="Text">use-case-4</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/open">
+      <compilation-unit name="use-case-4">
+        <output-dir compare="Text">use-case-4</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/closed">
+      <compilation-unit name="with-3-level-record-path">
+        <output-dir compare="Text">with-3-level-record-path</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/open">
+      <compilation-unit name="with-3-level-record-path">
+        <output-dir compare="Text">with-3-level-record-path</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/closed">
+      <compilation-unit name="with-composite-sk">
+        <output-dir compare="Text">with-composite-sk</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/closed">
+      <compilation-unit name="with-composite-pk">
+        <output-dir compare="Text">with-composite-pk</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/closed">
+      <compilation-unit name="with-filter-fields">
+        <output-dir compare="Text">with-filter-fields</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-unnest-queries/open">
+      <compilation-unit name="using-feed-new-index">
+        <output-dir compare="Text">using-feed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-quantified-queries">
+      <compilation-unit name="use-case-1">
+        <output-dir compare="Text">use-case-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-quantified-queries">
+      <compilation-unit name="use-case-2">
+        <output-dir compare="Text">use-case-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-quantified-queries">
+      <compilation-unit name="use-case-3">
+        <output-dir compare="Text">use-case-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-quantified-queries">
+      <compilation-unit name="with-composite-pk">
+        <output-dir compare="Text">with-composite-pk</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array-index/select-quantified-queries">
+      <compilation-unit name="with-open-index">
+        <output-dir compare="Text">with-open-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="cell-aggregation-with-filtering">
+        <output-dir compare="Text">cell-aggregation-with-filtering</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="outer_join_01">
+        <output-dir compare="Text">outer_join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="index_01">
+        <output-dir compare="Text">index_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q45">
+        <output-dir compare="Text">q45</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q10_returned_item">
+        <output-dir compare="Text">q10_returned_item</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q10_returned_item_int64">
+        <output-dir compare="Text">q10_returned_item_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q12_shipping">
+        <output-dir compare="Text">q12_shipping</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q14_promotion_effect">
+        <output-dir compare="Text">q14_promotion_effect</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q15_top_supplier">
+        <output-dir compare="Text">q15_top_supplier</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q01_pricing_summary_report_nt">
+        <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q03_shipping_priority_nt">
+        <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q04_order_priority">
+        <output-dir compare="Text">q04_order_priority</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q04_order_priority_with_nodegroup">
+        <output-dir compare="Text">q04_order_priority</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q05_local_supplier_volume">
+        <output-dir compare="Text">q05_local_supplier_volume</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q06_forecast_revenue_change">
+        <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q07_volume_shipping">
+        <output-dir compare="Text">q07_volume_shipping</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q08_national_market_share">
+        <output-dir compare="Text">q08_national_market_share</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue810">
+        <output-dir compare="Text">query-issue810</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue810-2">
+        <output-dir compare="Text">query-issue810-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue810-3">
+        <output-dir compare="Text">query-issue810-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue827-2">
+        <output-dir compare="Text">query-issue827-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-btree-sidx1">
+        <output-dir compare="Text">probe-pidx-with-join-btree-sidx1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-btree-sidx2">
+        <output-dir compare="Text">probe-pidx-with-join-btree-sidx2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-join-btree-sidx3-idxonly">
+        <output-dir compare="Text">probe-pidx-join-btree-sidx3-idxonly</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-rtree-sidx2">
+        <output-dir compare="Text">probe-pidx-with-join-rtree-sidx2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-invidx-sidx1">
+        <output-dir compare="Text">probe-pidx-with-join-invidx-sidx1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-invidx-sidx2">
+        <output-dir compare="Text">probe-pidx-with-join-invidx-sidx2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-idxonly-join-btree-pidx1">
+        <output-dir compare="Text">probe-sidx-btree-idxonly-join-btree-pidx1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-idxonly-join-btree-sidx1-idxonly">
+        <output-dir compare="Text">probe-sidx-btree-idxonly-join-btree-sidx1-idxonly</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-non-idxonly-join-btree-pidx1">
+        <output-dir compare="Text">probe-sidx-btree-non-idxonly-join-btree-pidx1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-non-idxonly-join-btree-sidx1-idxonly">
+        <output-dir compare="Text">probe-sidx-btree-non-idxonly-join-btree-sidx1-idxonly</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tinysocial">
+      <compilation-unit name="tinysocial-suite">
+        <output-dir compare="Text">tinysocial-suite</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tinysocial">
+      <compilation-unit name="tinysocial-suite-open">
+        <output-dir compare="Text">tinysocial-suite-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="load-with-secondary-btree">
+        <output-dir compare="Text">load-with-secondary-btree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="load-with-secondary-btree-index-only">
+        <output-dir compare="Text">load-with-secondary-btree-index-only</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="load-with-secondary-inverted-word">
+        <output-dir compare="Text">load-with-secondary-inverted-word</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="load-with-secondary-rtree">
+        <output-dir compare="Text">load-with-secondary-rtree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="insert-with-correlated-secondary-btree">
+        <output-dir compare="Text">insert-with-secondary-btree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="union_opt_1">
+        <output-dir compare="Text">union_opt_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="primary-correlated-secondary-btree">
+        <output-dir compare="Text">primary-secondary-btree</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>
+</test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_profiled.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_profiled.xml
new file mode 100644
index 0000000..c22d6d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_profiled.xml
@@ -0,0 +1,15880 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<!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">
+        <!ENTITY TemporalQueries SYSTEM "queries_sqlpp/temporal/TemporalQueries.xml">
+        ]>
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp" SourceLocation="true">
+  &ObjectsQueries;
+  &AsyncDeferredQueries;
+  &GeoQueries;
+  &TemporalQueries;
+  <test-group name="api">
+    <test-case FilePath="api">
+      <compilation-unit name="compileonly">
+        <output-dir compare="Text">compileonly</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
+      <compilation-unit name="multiple-param-values">
+        <output-dir compare="Text">multiple-param-values</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
+      <compilation-unit name="readonly-request">
+        <output-dir compare="Text">readonly-request</output-dir>
+        <expected-error>ASX0044: CREATE_DATAVERSE statement is not supported in read-only mode</expected-error>
+        <expected-error>ASX0044: DATAVERSE_DROP statement is not supported in read-only mode</expected-error>
+        <expected-error>ASX0044: DATASET_DECL statement is not supported in read-only mode</expected-error>
+        <expected-error>ASX0044: DATASET_DROP statement is not supported in read-only mode</expected-error>
+        <expected-error>ASX0044: CREATE_INDEX statement is not supported in read-only mode</expected-error>
+        <expected-error>ASX0044: INDEX_DROP statement is not supported in read-only mode</expected-error>
+        <expected-error>ASX0044: INSERT statement is not supported in read-only mode</expected-error>
+        <expected-error>ASX0044: UPSERT statement is not supported in read-only mode</expected-error>
+        <expected-error>ASX0044: DELETE statement is not supported in read-only mode</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api" check-warnings="true">
+      <compilation-unit name="request-dataverse">
+        <output-dir compare="Text">request-dataverse</output-dir>
+        <expected-warn>ASX1063: Cannot find dataverse with name testUnknown</expected-warn>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
+      <compilation-unit name="request-param-validation">
+        <output-dir compare="Text">request-param-validation</output-dir>
+        <expected-error>Invalid value for parameter 'format': foo</expected-error>
+        <expected-error>Invalid value for parameter 'pretty': bar</expected-error>
+        <expected-error>Invalid value for parameter 'plan-format': blah</expected-error>
+        <expected-error>Invalid value for parameter 'max-result-reads': foo</expected-error>
+        <expected-error>Invalid value for parameter 'max-result-reads': 9999999999999999999999999999999999999999</expected-error>
+        <expected-error>Invalid value for parameter 'max-warnings': baz</expected-error>
+        <expected-error>Invalid value for parameter 'max-warnings': 1.5</expected-error>
+        <expected-error>Invalid value for parameter 'mode': asyn</expected-error>
+        <expected-error>Invalid value for parameter 'timeout': 12</expected-error>
+        <expected-error>Invalid value for parameter 'args': 12</expected-error>
+        <expected-error>Unable to process JSON content in request</expected-error>
+        <expected-error>Unable to process JSON content in request</expected-error>
+        <expected-error>Invalid value for parameter 'format': foo</expected-error>
+        <expected-error>Invalid value for parameter 'pretty': bar</expected-error>
+        <expected-error>Invalid value for parameter 'plan-format': blah</expected-error>
+        <expected-error>Invalid value for parameter 'max-result-reads': foo</expected-error>
+        <expected-error>Invalid value for parameter 'max-warnings': baz</expected-error>
+        <expected-error>Invalid value for parameter 'mode': asyn</expected-error>
+        <expected-error>Invalid value for parameter 'args': 12</expected-error>
+        <expected-error>Unable to process JSON content in request</expected-error>
+        <expected-error>Unable to process JSON content in request</expected-error>
+        <expected-error>Invalid value for parameter 'profile': true</expected-error>
+        <expected-error>Invalid value for parameter 'profile': true</expected-error>
+        <expected-error>Invalid value for parameter 'profile': foo</expected-error>
+        <expected-error>Invalid value for parameter 'profile': foo</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
+      <compilation-unit name="request-param-validation-400-BAD">
+        <output-dir compare="Text">request-param-validation-400-BAD</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
+      <compilation-unit name="request-param">
+        <output-dir compare="Text">request-param</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
+      <compilation-unit name="format-param-in-accept-01">
+        <output-dir compare="Text">format-param-in-accept-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
+      <compilation-unit name="format-param-in-accept-02">
+        <output-dir compare="Clean-JSON">format-param-in-accept-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
+      <compilation-unit name="format-param-in-accept-03">
+        <output-dir compare="Lossless-JSON">format-param-in-accept-03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
+      <compilation-unit name="format-param-in-accept-04">
+        <output-dir compare="Text">format-param-in-accept-04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
+      <compilation-unit name="format-param-in-accept-05">
+        <output-dir compare="AST">format-param-in-accept-05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
+      <compilation-unit name="get-non-query">
+        <output-dir compare="Text">get-non-query</output-dir>
+        <expected-error>CREATE_DATAVERSE statement is not supported in read-only mode</expected-error>
+        <expected-error>TYPE_DECL statement is not supported in read-only mode</expected-error>
+        <expected-error>CREATE_FUNCTION statement is not supported in read-only mode</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
+      <compilation-unit name="get-query">
+        <output-dir compare="Text">get-query</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
+      <compilation-unit name="post-non-query">
+        <output-dir compare="Text">post-non-query</output-dir>
+        <expected-error>TYPE_DECL statement is not supported in read-only mode</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
+      <compilation-unit name="signature">
+        <output-dir compare="Text">signature</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
+      <compilation-unit name="ignore-body-for-get">
+        <output-dir compare="Text">ignore-body-for-get</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="flwor">
+    <test-case FilePath="flwor">
+      <compilation-unit name="at00">
+        <output-dir compare="Text">at00</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="at01">
+        <output-dir compare="Text">at01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="at02">
+        <output-dir compare="Text">at02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="at03">
+        <output-dir compare="Text">at03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="at04">
+        <output-dir compare="Text">at04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="at05">
+        <output-dir compare="Text">at05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="at06">
+        <output-dir compare="Text">at06</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="at07">
+        <output-dir compare="Text">at07</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="order-by-13">
+        <output-dir compare="Text">order-by-13</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="query-issue550">
+        <output-dir compare="Text">query-issue550</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="query-ASTERIXDB-883">
+        <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="query-ASTERIXDB-1576">
+        <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
+        <expected-error>Duplicate alias definitions: samptable1</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="query-ASTERIXDB-1576-2">
+        <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
+        <expected-error>Duplicate alias definitions: s2</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="query-ASTERIXDB-1576-3">
+        <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
+        <expected-error>Duplicate alias definitions: s1</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor" check-warnings="true">
+      <compilation-unit name="query-ASTERIXDB-2446">
+        <output-dir compare="Text">query-ASTERIXDB-2446</output-dir>
+        <expected-error>ASX0013: Duplicate field name 'a'</expected-error>
+        <expected-warn>Duplicate field name 'c' (in line 28, at column 84)</expected-warn>
+        <expected-warn>Duplicate field name 'e' (in line 28, at column 116)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="query-ASTERIXDB-2446-2">
+        <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
+        <expected-error>ASX0013: Duplicate field name 'a' (in line 27, at column 20)</expected-error>
+        <expected-error>ASX0013: Duplicate field name 'b' (in line 27, at column 20)</expected-error>
+        <expected-error>ASX0013: Duplicate field name 'c' (in line 27, at column 11)</expected-error>
+        <expected-error>ASX0013: Duplicate field name 'd' (in line 27, at column 11)</expected-error>
+        <expected-error>ASX0013: Duplicate field name 'e' (in line 27, at column 14)</expected-error>
+        <expected-error>ASX0013: Duplicate field name 'f' (in line 27, at column 11)</expected-error>
+        <expected-error>ASX0013: Duplicate field name 'g' (in line 27, at column 11)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="select-let">
+        <output-dir compare="Text">select-let</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="let33">
+        <output-dir compare="Text">let33</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="at00">
+        <output-dir compare="Text">at00</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--test-case FilePath="flwor">
+      <compilation-unit name="query-ASTERIXDB-1463">
+        <output-dir compare="Text">query-ASTERIXDB-1485</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="flwor">
+      <compilation-unit name="query-ASTERIXDB-1485">
+        <output-dir compare="Text">query-ASTERIXDB-1485</output-dir>
+      </compilation-unit>
+    </test-case!-->
+  </test-group>
+  <test-group name="sorting">
+    <test-case FilePath="sorting">
+      <compilation-unit name="arrays">
+        <output-dir compare="Text">arrays</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sorting">
+      <compilation-unit name="range_hint">
+        <output-dir compare="Text">range_hint</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sorting">
+      <compilation-unit name="records">
+        <output-dir compare="Text">records</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="explain">
+    <test-case FilePath="explain">
+      <compilation-unit name="explain_simple">
+        <output-dir compare="Text">explain_simple</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="explain">
+      <compilation-unit name="explain_caps">
+        <output-dir compare="Text">explain_simple</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="explain">
+      <compilation-unit name="explain_pretty">
+        <parameter name="pretty" value="true" />
+        <output-dir compare="Text">explain_simple</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="explain">
+      <compilation-unit name="explain_object_constructor-01">
+        <output-dir compare="Text">explain_object_constructor-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="explain">
+      <compilation-unit name="explain_object_constructor-02">
+        <output-dir compare="Text">explain_object_constructor-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="explain">
+      <compilation-unit name="explain_object_constructor-03">
+        <output-dir compare="Text">explain_object_constructor-03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="explain">
+      <compilation-unit name="explain_field_access">
+        <output-dir compare="Text">explain_field_access</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="explain">
+      <compilation-unit name="explain_field_access_closed">
+        <output-dir compare="Text">explain_field_access_closed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="explain">
+      <compilation-unit name="explain_negative">
+        <output-dir compare="Text">explain_simple</output-dir>
+        <expected-error>ASX1001: Syntax error: EXPLAIN is not supported for this kind of statement</expected-error>
+        <expected-error>ASX1001: Syntax error: EXPLAIN is not supported for this kind of statement</expected-error>
+        <expected-error>ASX1001: Syntax error: EXPLAIN is not supported for this kind of statement</expected-error>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="aggregate">
+    <test-case FilePath="aggregate">
+      <compilation-unit name="issue531_string_min_max">
+        <output-dir compare="Text">issue531_string_min_max</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="agg_null">
+        <output-dir compare="Text">agg_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="agg_null_rec">
+        <output-dir compare="Text">agg_null_rec</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="agg_null_rec_1">
+        <output-dir compare="Text">agg_null_rec_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="agg_number_rec">
+        <output-dir compare="Text">agg_number_rec</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate" check-warnings="true">
+      <compilation-unit name="avg_mixed">
+        <output-dir compare="Text">avg_mixed</output-dir>
+        <expected-warn>Unsupported type: agg-avg cannot process input type string (in line 26, at column 16)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate" check-warnings="true">
+      <compilation-unit name="serial_avg_mixed">
+        <output-dir compare="Text">serial_avg_mixed</output-dir>
+        <expected-warn>Unsupported type: agg-avg cannot process input type string (in line 29, at column 39)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="group_only">
+        <output-dir compare="Text">group_only</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="min_mixed">
+        <output-dir compare="Text">min_mixed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_mixed">
+        <output-dir compare="Text">stddev_mixed</output-dir>
+        <expected-error>Type incompatibility: function agg-stddev_samp gets incompatible input values: string and float</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_stddev_mixed">
+        <output-dir compare="Text">serial_stddev_mixed</output-dir>
+        <expected-error>Type incompatibility: function agg-stddev_samp gets incompatible input values: string and float</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_pop_mixed">
+        <output-dir compare="Text">stddev_pop_mixed</output-dir>
+        <expected-error>Type incompatibility: function agg-stddev_pop gets incompatible input values: string and float</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_stddev_pop_mixed">
+        <output-dir compare="Text">serial_stddev_pop_mixed</output-dir>
+        <expected-error>Type incompatibility: function agg-stddev_pop gets incompatible input values: string and float</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate" check-warnings="true">
+      <compilation-unit name="sum/sum_mixed">
+        <output-dir compare="Text">sum/sum_mixed</output-dir>
+        <expected-warn>Unsupported type: agg-sum cannot process input type string (in line 27, at column 16)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate" check-warnings="true">
+      <compilation-unit name="sum/serial_sum_mixed">
+        <output-dir compare="Text">sum/serial_sum_mixed</output-dir>
+        <expected-warn>Unsupported type: agg-sum cannot process input type string (in line 29, at column 39)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_mixed">
+        <output-dir compare="Text">var_mixed</output-dir>
+        <expected-error>Type incompatibility: function agg-var_samp gets incompatible input values: string and float</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_var_mixed">
+        <output-dir compare="Text">serial_var_mixed</output-dir>
+        <expected-error>Type incompatibility: function agg-var_samp gets incompatible input values: string and float</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_pop_mixed">
+        <output-dir compare="Text">var_pop_mixed</output-dir>
+        <expected-error>Type incompatibility: function agg-var_pop gets incompatible input values: string and float</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_var_pop_mixed">
+        <output-dir compare="Text">serial_var_pop_mixed</output-dir>
+        <expected-error>Type incompatibility: function agg-var_pop gets incompatible input values: string and float</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="kurtosis_mixed">
+        <output-dir compare="Text">kurtosis_mixed</output-dir>
+        <expected-error>Type incompatibility: function agg-kurtosis gets incompatible input values: string and float</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_kurtosis_mixed">
+        <output-dir compare="Text">serial_kurtosis_mixed</output-dir>
+        <expected-error>Type incompatibility: function agg-kurtosis gets incompatible input values: string and float</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="skewness_mixed">
+        <output-dir compare="Text">skewness_mixed</output-dir>
+        <expected-error>Type incompatibility: function agg-skewness gets incompatible input values: string and float</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_skewness_mixed">
+        <output-dir compare="Text">serial_skewness_mixed</output-dir>
+        <expected-error>Type incompatibility: function agg-skewness gets incompatible input values: string and float</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="agg_number">
+        <output-dir compare="Text">agg_number</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="issue425_min_hetero_list_1">
+        <output-dir compare="Text">issue425_min_hetero_list_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="issue425_min_hetero_list">
+        <output-dir compare="Text">issue425_min_hetero_list</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/issue425_sum_hetero_list_1">
+        <output-dir compare="Text">sum/issue425_sum_hetero_list_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/issue425_sum_hetero_list">
+        <output-dir compare="Text">sum/issue425_sum_hetero_list</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="query-issue400">
+        <output-dir compare="Text">query-issue400</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="issue395">
+        <output-dir compare="Text">issue395</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="issue412_0">
+        <output-dir compare="Text">issue412_0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="issue412_1">
+        <output-dir compare="Text">issue412_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="avg_double">
+        <output-dir compare="Text">avg_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="avg_double_null">
+        <output-dir compare="Text">avg_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="avg_empty_01">
+        <output-dir compare="Text">avg_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="avg_empty_02">
+        <output-dir compare="Text">avg_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="avg_float">
+        <output-dir compare="Text">avg_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="avg_float_null">
+        <output-dir compare="Text">avg_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="avg_int16">
+        <output-dir compare="Text">avg_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="avg_int16_null">
+        <output-dir compare="Text">avg_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="avg_int32">
+        <output-dir compare="Text">avg_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="avg_int32_null">
+        <output-dir compare="Text">avg_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="avg_int64">
+        <output-dir compare="Text">avg_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="avg_int64_null">
+        <output-dir compare="Text">avg_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="avg_int8">
+        <output-dir compare="Text">avg_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="avg_int8_null">
+        <output-dir compare="Text">avg_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="avg_distinct">
+        <output-dir compare="Text">avg_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="count_01">
+        <output-dir compare="Text">count_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="count_dataset">
+        <output-dir compare="Text">count_dataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="count_empty_01">
+        <output-dir compare="Text">count_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="count_empty_02">
+        <output-dir compare="Text">count_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="count_null">
+        <output-dir compare="Text">count_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="count_distinct">
+        <output-dir compare="Text">count_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="kurtosis_double">
+        <output-dir compare="Text">kurtosis_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="kurtosis_double_null">
+        <output-dir compare="Text">kurtosis_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="kurtosis_empty_01">
+        <output-dir compare="Text">kurtosis_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="kurtosis_empty_02">
+        <output-dir compare="Text">kurtosis_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="kurtosis_float">
+        <output-dir compare="Text">kurtosis_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="kurtosis_float_null">
+        <output-dir compare="Text">kurtosis_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="kurtosis_int16">
+        <output-dir compare="Text">kurtosis_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="kurtosis_int16_null">
+        <output-dir compare="Text">kurtosis_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="kurtosis_int32">
+        <output-dir compare="Text">kurtosis_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="kurtosis_int32_null">
+        <output-dir compare="Text">kurtosis_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="kurtosis_int64">
+        <output-dir compare="Text">kurtosis_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="kurtosis_int64_null">
+        <output-dir compare="Text">kurtosis_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="kurtosis_int8">
+        <output-dir compare="Text">kurtosis_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="kurtosis_int8_null">
+        <output-dir compare="Text">kurtosis_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="kurtosis_distinct">
+        <output-dir compare="Text">kurtosis_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="max_empty_01">
+        <output-dir compare="Text">max_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="max_empty_02">
+        <output-dir compare="Text">max_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="max_distinct">
+        <output-dir compare="Text">max_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="min_empty_01">
+        <output-dir compare="Text">min_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="min_empty_02">
+        <output-dir compare="Text">min_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="min_distinct">
+        <output-dir compare="Text">min_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_avg">
+        <output-dir compare="Text">scalar_avg</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_avg_empty">
+        <output-dir compare="Text">scalar_avg_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_avg_null">
+        <output-dir compare="Text">scalar_avg_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_count">
+        <output-dir compare="Text">scalar_count</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_count_empty">
+        <output-dir compare="Text">scalar_count_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_count_null">
+        <output-dir compare="Text">scalar_count_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_kurtosis">
+        <output-dir compare="Text">scalar_kurtosis</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_kurtosis_empty">
+        <output-dir compare="Text">scalar_kurtosis_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_kurtosis_null">
+        <output-dir compare="Text">scalar_kurtosis_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_max">
+        <output-dir compare="Text">scalar_max</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_max_empty">
+        <output-dir compare="Text">scalar_max_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_max_null">
+        <output-dir compare="Text">scalar_max_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_min">
+        <output-dir compare="Text">scalar_min</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_min_empty">
+        <output-dir compare="Text">scalar_min_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_min_null">
+        <output-dir compare="Text">scalar_min_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_skewness">
+        <output-dir compare="Text">scalar_skewness</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_skewness_empty">
+        <output-dir compare="Text">scalar_skewness_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_skewness_null">
+        <output-dir compare="Text">scalar_skewness_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_stddev">
+        <output-dir compare="Text">scalar_stddev</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_stddev_empty">
+        <output-dir compare="Text">scalar_stddev_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_stddev_null">
+        <output-dir compare="Text">scalar_stddev_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/scalar_sum">
+        <output-dir compare="Text">sum/scalar_sum</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/scalar_sum_empty">
+        <output-dir compare="Text">sum/scalar_sum_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/scalar_sum_null">
+        <output-dir compare="Text">sum/scalar_sum_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/scalar_sum_type">
+        <output-dir compare="Text">sum/scalar_sum_type</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_var">
+        <output-dir compare="Text">scalar_var</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_var_empty">
+        <output-dir compare="Text">scalar_var_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="scalar_var_null">
+        <output-dir compare="Text">scalar_var_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_avg_empty">
+        <output-dir compare="Text">serial_avg_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_avg_int8">
+        <output-dir compare="Text">serial_avg_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_avg_int8_null">
+        <output-dir compare="Text">serial_avg_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_avg_int16">
+        <output-dir compare="Text">serial_avg_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_avg_int16_null">
+        <output-dir compare="Text">serial_avg_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_avg_int32">
+        <output-dir compare="Text">serial_avg_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_avg_int32_null">
+        <output-dir compare="Text">serial_avg_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_avg_int64">
+        <output-dir compare="Text">serial_avg_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_avg_int64_null">
+        <output-dir compare="Text">serial_avg_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_avg_float">
+        <output-dir compare="Text">serial_avg_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_avg_float_null">
+        <output-dir compare="Text">serial_avg_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_avg_double">
+        <output-dir compare="Text">serial_avg_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_avg_double_null">
+        <output-dir compare="Text">serial_avg_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_kurtosis_double">
+        <output-dir compare="Text">serial_kurtosis_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_kurtosis_double_null">
+        <output-dir compare="Text">serial_kurtosis_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_kurtosis_empty">
+        <output-dir compare="Text">serial_kurtosis_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_kurtosis_float">
+        <output-dir compare="Text">serial_kurtosis_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_kurtosis_float_null">
+        <output-dir compare="Text">serial_kurtosis_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_kurtosis_int16">
+        <output-dir compare="Text">serial_kurtosis_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_kurtosis_int16_null">
+        <output-dir compare="Text">serial_kurtosis_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_kurtosis_int32">
+        <output-dir compare="Text">serial_kurtosis_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_kurtosis_int32_null">
+        <output-dir compare="Text">serial_kurtosis_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_kurtosis_int64">
+        <output-dir compare="Text">serial_kurtosis_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_kurtosis_int64_null">
+        <output-dir compare="Text">serial_kurtosis_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_kurtosis_int8">
+        <output-dir compare="Text">serial_kurtosis_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_kurtosis_int8_null">
+        <output-dir compare="Text">serial_kurtosis_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_skewness_double">
+        <output-dir compare="Text">serial_skewness_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_skewness_double_null">
+        <output-dir compare="Text">serial_skewness_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_skewness_empty">
+        <output-dir compare="Text">serial_skewness_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_skewness_float">
+        <output-dir compare="Text">serial_skewness_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_skewness_float_null">
+        <output-dir compare="Text">serial_skewness_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_skewness_int16">
+        <output-dir compare="Text">serial_skewness_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_skewness_int16_null">
+        <output-dir compare="Text">serial_skewness_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_skewness_int32">
+        <output-dir compare="Text">serial_skewness_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_skewness_int32_null">
+        <output-dir compare="Text">serial_skewness_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_skewness_int64">
+        <output-dir compare="Text">serial_skewness_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_skewness_int64_null">
+        <output-dir compare="Text">serial_skewness_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_skewness_int8">
+        <output-dir compare="Text">serial_skewness_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_skewness_int8_null">
+        <output-dir compare="Text">serial_skewness_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_stddev_double">
+        <output-dir compare="Text">serial_stddev_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_stddev_double_null">
+        <output-dir compare="Text">serial_stddev_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_stddev_empty">
+        <output-dir compare="Text">serial_stddev_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_stddev_float">
+        <output-dir compare="Text">serial_stddev_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_stddev_float_null">
+        <output-dir compare="Text">serial_stddev_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_stddev_int16">
+        <output-dir compare="Text">serial_stddev_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_stddev_int16_null">
+        <output-dir compare="Text">serial_stddev_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_stddev_int32">
+        <output-dir compare="Text">serial_stddev_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_stddev_int32_null">
+        <output-dir compare="Text">serial_stddev_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_stddev_int64">
+        <output-dir compare="Text">serial_stddev_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_stddev_int64_null">
+        <output-dir compare="Text">serial_stddev_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_stddev_int8">
+        <output-dir compare="Text">serial_stddev_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_stddev_int8_null">
+        <output-dir compare="Text">serial_stddev_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/serial_sum_double">
+        <output-dir compare="Text">sum/serial_sum_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/serial_sum_double_null">
+        <output-dir compare="Text">sum/serial_sum_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/serial_sum_empty">
+        <output-dir compare="Text">sum/serial_sum_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/serial_sum_float">
+        <output-dir compare="Text">sum/serial_sum_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/serial_sum_float_null">
+        <output-dir compare="Text">sum/serial_sum_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/serial_sum_int16">
+        <output-dir compare="Text">sum/serial_sum_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/serial_sum_int16_null">
+        <output-dir compare="Text">sum/serial_sum_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/serial_sum_int32">
+        <output-dir compare="Text">sum/serial_sum_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/serial_sum_int32_null">
+        <output-dir compare="Text">sum/serial_sum_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/serial_sum_int64">
+        <output-dir compare="Text">sum/serial_sum_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/serial_sum_int64_null">
+        <output-dir compare="Text">sum/serial_sum_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/serial_sum_int64_overflow">
+        <output-dir compare="Text">sum/serial_sum_int64_overflow</output-dir>
+        <expected-error>Overflow in agg-sum</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/serial_sum_int8">
+        <output-dir compare="Text">sum/serial_sum_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/serial_sum_int8_null">
+        <output-dir compare="Text">sum/serial_sum_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_var_double">
+        <output-dir compare="Text">serial_var_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_var_double_null">
+        <output-dir compare="Text">serial_var_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_var_empty">
+        <output-dir compare="Text">serial_var_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_var_float">
+        <output-dir compare="Text">serial_var_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_var_float_null">
+        <output-dir compare="Text">serial_var_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_var_int16">
+        <output-dir compare="Text">serial_var_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_var_int16_null">
+        <output-dir compare="Text">serial_var_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_var_int32">
+        <output-dir compare="Text">serial_var_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_var_int32_null">
+        <output-dir compare="Text">serial_var_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_var_int64">
+        <output-dir compare="Text">serial_var_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_var_int64_null">
+        <output-dir compare="Text">serial_var_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_var_int8">
+        <output-dir compare="Text">serial_var_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="serial_var_int8_null">
+        <output-dir compare="Text">serial_var_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="skewness_double">
+        <output-dir compare="Text">skewness_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="skewness_double_null">
+        <output-dir compare="Text">skewness_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="skewness_empty_01">
+        <output-dir compare="Text">skewness_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="skewness_empty_02">
+        <output-dir compare="Text">skewness_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="skewness_float">
+        <output-dir compare="Text">skewness_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="skewness_float_null">
+        <output-dir compare="Text">skewness_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="skewness_int16">
+        <output-dir compare="Text">skewness_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="skewness_int16_null">
+        <output-dir compare="Text">skewness_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="skewness_int32">
+        <output-dir compare="Text">skewness_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="skewness_int32_null">
+        <output-dir compare="Text">skewness_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="skewness_int64">
+        <output-dir compare="Text">skewness_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="skewness_int64_null">
+        <output-dir compare="Text">skewness_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="skewness_int8">
+        <output-dir compare="Text">skewness_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="skewness_int8_null">
+        <output-dir compare="Text">skewness_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="skewness_distinct">
+        <output-dir compare="Text">skewness_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_double">
+        <output-dir compare="Text">stddev_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_double_null">
+        <output-dir compare="Text">stddev_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_empty_01">
+        <output-dir compare="Text">stddev_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_empty_02">
+        <output-dir compare="Text">stddev_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_float">
+        <output-dir compare="Text">stddev_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_float_null">
+        <output-dir compare="Text">stddev_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_int16">
+        <output-dir compare="Text">stddev_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_int16_null">
+        <output-dir compare="Text">stddev_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_int32">
+        <output-dir compare="Text">stddev_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_int32_null">
+        <output-dir compare="Text">stddev_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_int64">
+        <output-dir compare="Text">stddev_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_int64_null">
+        <output-dir compare="Text">stddev_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_int8">
+        <output-dir compare="Text">stddev_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_int8_null">
+        <output-dir compare="Text">stddev_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_distinct">
+        <output-dir compare="Text">stddev_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_misc">
+        <output-dir compare="Text">stddev_misc</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="stddev_pop_misc">
+        <output-dir compare="Text">stddev_pop_misc</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_double">
+        <output-dir compare="Text">sum/sum_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_double_null">
+        <output-dir compare="Text">sum/sum_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_empty_01">
+        <output-dir compare="Text">sum/sum_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_empty_02">
+        <output-dir compare="Text">sum/sum_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_float">
+        <output-dir compare="Text">sum/sum_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_float_null">
+        <output-dir compare="Text">sum/sum_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_int16">
+        <output-dir compare="Text">sum/sum_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_int16_null">
+        <output-dir compare="Text">sum/sum_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_int32">
+        <output-dir compare="Text">sum/sum_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_int32_null">
+        <output-dir compare="Text">sum/sum_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_int64">
+        <output-dir compare="Text">sum/sum_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_int64_null">
+        <output-dir compare="Text">sum/sum_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_int64_null">
+        <output-dir compare="Text">sum/sum_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_int64_overflow">
+        <output-dir compare="Text">sum/sum_int64_overflow</output-dir>
+        <expected-error>Overflow in agg-sum</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_int8">
+        <output-dir compare="Text">sum/sum_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_int8_null">
+        <output-dir compare="Text">sum/sum_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_null-with-pred">
+        <output-dir compare="Text">sum/sum_null-with-pred</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_numeric_null">
+        <output-dir compare="Text">sum/sum_numeric_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="sum/sum_distinct">
+        <output-dir compare="Text">sum/sum_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_double">
+        <output-dir compare="Text">var_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_double_null">
+        <output-dir compare="Text">var_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_empty_01">
+        <output-dir compare="Text">var_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_empty_02">
+        <output-dir compare="Text">var_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_float">
+        <output-dir compare="Text">var_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_float_null">
+        <output-dir compare="Text">var_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_int16">
+        <output-dir compare="Text">var_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_int16_null">
+        <output-dir compare="Text">var_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_int32">
+        <output-dir compare="Text">var_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_int32_null">
+        <output-dir compare="Text">var_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_int64">
+        <output-dir compare="Text">var_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_int64_null">
+        <output-dir compare="Text">var_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_int8">
+        <output-dir compare="Text">var_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_int8_null">
+        <output-dir compare="Text">var_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_distinct">
+        <output-dir compare="Text">var_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_misc">
+        <output-dir compare="Text">var_misc</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="var_pop_misc">
+        <output-dir compare="Text">var_pop_misc</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="min_max_arrays">
+        <output-dir compare="Text">min_max_arrays</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="aggregate-sql">
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="min_max_arrays">
+        <output-dir compare="Text">min_max_arrays</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="issue531_string_min_max">
+        <output-dir compare="Text">issue531_string_min_max</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="agg_null">
+        <output-dir compare="Text">agg_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="agg_null_rec">
+        <output-dir compare="Text">agg_null_rec</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="agg_null_rec_1">
+        <output-dir compare="Text">agg_null_rec_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="agg_number_rec">
+        <output-dir compare="Text">agg_number_rec</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql" check-warnings="true">
+      <compilation-unit name="avg_mixed">
+        <output-dir compare="Text">avg_mixed</output-dir>
+        <expected-warn>Unsupported type: agg-avg cannot process input type string (in line 26, at column 16)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql" check-warnings="true">
+      <compilation-unit name="serial_avg_mixed">
+        <output-dir compare="Text">serial_avg_mixed</output-dir>
+        <expected-warn>Unsupported type: agg-avg cannot process input type string (in line 29, at column 38)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="min_mixed">
+        <output-dir compare="Text">min_mixed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_stddev_mixed">
+        <output-dir compare="Text">serial_stddev_mixed</output-dir>
+        <expected-error>Invalid item type: function agg-stddev_samp cannot process item type string in an input array (or multiset)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_mixed">
+        <output-dir compare="Text">stddev_mixed</output-dir>
+        <expected-error>Type incompatibility: function agg-stddev_samp gets incompatible input values: string and float</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_stddev_pop_mixed">
+        <output-dir compare="Text">serial_stddev_pop_mixed</output-dir>
+        <expected-error>Invalid item type: function agg-stddev_pop cannot process item type string in an input array (or multiset)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_pop_mixed">
+        <output-dir compare="Text">stddev_pop_mixed</output-dir>
+        <expected-error>Type incompatibility: function agg-stddev_pop gets incompatible input values: string and float</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql" check-warnings="true">
+      <compilation-unit name="sum/sum_mixed">
+        <output-dir compare="Text">sum/sum_mixed</output-dir>
+        <expected-warn>Unsupported type: agg-sum cannot process input type string (in line 27, at column 16)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql" check-warnings="true">
+      <compilation-unit name="sum/serial_sum_mixed">
+        <output-dir compare="Text">sum/serial_sum_mixed</output-dir>
+        <expected-warn>Unsupported type: agg-sum cannot process input type string (in line 29, at column 38)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_var_mixed">
+        <output-dir compare="Text">serial_var_mixed</output-dir>
+        <expected-error>Invalid item type: function agg-var_samp cannot process item type string in an input array (or multiset)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_mixed">
+        <output-dir compare="Text">var_mixed</output-dir>
+        <expected-error>Type incompatibility: function agg-var_samp gets incompatible input values: string and float</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_var_pop_mixed">
+        <output-dir compare="Text">serial_var_pop_mixed</output-dir>
+        <expected-error>Invalid item type: function agg-var_pop cannot process item type string in an input array (or multiset)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_pop_mixed">
+        <output-dir compare="Text">var_pop_mixed</output-dir>
+        <expected-error>Type incompatibility: function agg-var_pop gets incompatible input values: string and float</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_skewness_mixed">
+        <output-dir compare="Text">serial_skewness_mixed</output-dir>
+        <expected-error>Invalid item type: function agg-skewness cannot process item type string in an input array (or multiset)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="skewness_mixed">
+        <output-dir compare="Text">skewness_mixed</output-dir>
+        <expected-error>Type incompatibility: function agg-skewness gets incompatible input values: string and float</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="agg_number">
+        <output-dir compare="Text">agg_number</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="issue425_min_hetero_list_1">
+        <output-dir compare="Text">issue425_min_hetero_list_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="issue425_min_hetero_list">
+        <output-dir compare="Text">issue425_min_hetero_list</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/issue425_sum_hetero_list_1">
+        <output-dir compare="Text">sum/issue425_sum_hetero_list_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/issue425_sum_hetero_list">
+        <output-dir compare="Text">sum/issue425_sum_hetero_list</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="query-issue400">
+        <output-dir compare="Text">query-issue400</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="issue395">
+        <output-dir compare="Text">issue395</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="issue412_0">
+        <output-dir compare="Text">issue412_0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="issue412_1">
+        <output-dir compare="Text">issue412_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="issue2348">
+        <output-dir compare="Text">issue2348</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="issue2411">
+        <output-dir compare="Text">issue2411</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="avg_double">
+        <output-dir compare="Text">avg_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="avg_double_null">
+        <output-dir compare="Text">avg_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="avg_empty_01">
+        <output-dir compare="Text">avg_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="avg_empty_02">
+        <output-dir compare="Text">avg_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="avg_float">
+        <output-dir compare="Text">avg_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="avg_float_null">
+        <output-dir compare="Text">avg_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="avg_int16">
+        <output-dir compare="Text">avg_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="avg_int16_null">
+        <output-dir compare="Text">avg_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="avg_int32">
+        <output-dir compare="Text">avg_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="avg_int32_null">
+        <output-dir compare="Text">avg_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="avg_int64">
+        <output-dir compare="Text">avg_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="avg_int64_null">
+        <output-dir compare="Text">avg_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="avg_int8">
+        <output-dir compare="Text">avg_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="avg_int8_null">
+        <output-dir compare="Text">avg_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="avg_distinct">
+        <output-dir compare="Text">avg_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="count_01">
+        <output-dir compare="Text">count_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="count_dataset">
+        <output-dir compare="Text">count_dataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="count_empty_01">
+        <output-dir compare="Text">count_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="count_empty_02">
+        <output-dir compare="Text">count_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="count_null">
+        <output-dir compare="Text">count_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="count_distinct">
+        <output-dir compare="Text">count_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="kurtosis_double">
+        <output-dir compare="Text">kurtosis_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="kurtosis_double_null">
+        <output-dir compare="Text">kurtosis_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="kurtosis_empty_01">
+        <output-dir compare="Text">kurtosis_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="kurtosis_empty_02">
+        <output-dir compare="Text">kurtosis_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="kurtosis_float">
+        <output-dir compare="Text">kurtosis_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="kurtosis_float_null">
+        <output-dir compare="Text">kurtosis_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="kurtosis_int16">
+        <output-dir compare="Text">kurtosis_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="kurtosis_int16_null">
+        <output-dir compare="Text">kurtosis_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="kurtosis_int32">
+        <output-dir compare="Text">kurtosis_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="kurtosis_int32_null">
+        <output-dir compare="Text">kurtosis_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="kurtosis_int64">
+        <output-dir compare="Text">kurtosis_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="kurtosis_int64_null">
+        <output-dir compare="Text">kurtosis_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="kurtosis_int8">
+        <output-dir compare="Text">kurtosis_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="kurtosis_int8_null">
+        <output-dir compare="Text">kurtosis_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="kurtosis_distinct">
+        <output-dir compare="Text">kurtosis_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="max_empty_01">
+        <output-dir compare="Text">max_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="max_empty_02">
+        <output-dir compare="Text">max_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="max_distinct">
+        <output-dir compare="Text">max_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="min_empty_01">
+        <output-dir compare="Text">min_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="min_empty_02">
+        <output-dir compare="Text">min_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="min_distinct">
+        <output-dir compare="Text">min_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_avg">
+        <output-dir compare="Text">scalar_avg</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_avg_empty">
+        <output-dir compare="Text">scalar_avg_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_avg_null">
+        <output-dir compare="Text">scalar_avg_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_count">
+        <output-dir compare="Text">scalar_count</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_count_empty">
+        <output-dir compare="Text">scalar_count_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_count_null">
+        <output-dir compare="Text">scalar_count_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_kurtosis">
+        <output-dir compare="Text">scalar_kurtosis</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_kurtosis_empty">
+        <output-dir compare="Text">scalar_kurtosis_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_kurtosis_null">
+        <output-dir compare="Text">scalar_kurtosis_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_max">
+        <output-dir compare="Text">scalar_max</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_max_empty">
+        <output-dir compare="Text">scalar_max_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_max_null">
+        <output-dir compare="Text">scalar_max_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_min">
+        <output-dir compare="Text">scalar_min</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_min_empty">
+        <output-dir compare="Text">scalar_min_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_min_null">
+        <output-dir compare="Text">scalar_min_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_skewness">
+        <output-dir compare="Text">scalar_skewness</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_skewness_empty">
+        <output-dir compare="Text">scalar_skewness_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_skewness_null">
+        <output-dir compare="Text">scalar_skewness_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_stddev">
+        <output-dir compare="Text">scalar_stddev</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_stddev_empty">
+        <output-dir compare="Text">scalar_stddev_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_stddev_null">
+        <output-dir compare="Text">scalar_stddev_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/scalar_sum">
+        <output-dir compare="Text">sum/scalar_sum</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/scalar_sum_empty">
+        <output-dir compare="Text">sum/scalar_sum_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/scalar_sum_null">
+        <output-dir compare="Text">sum/scalar_sum_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/scalar_sum_type">
+        <output-dir compare="Text">sum/scalar_sum_type</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_var">
+        <output-dir compare="Text">scalar_var</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_var_empty">
+        <output-dir compare="Text">scalar_var_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="scalar_var_null">
+        <output-dir compare="Text">scalar_var_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_avg_empty">
+        <output-dir compare="Text">serial_avg_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_avg_int8">
+        <output-dir compare="Text">serial_avg_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_avg_int8_null">
+        <output-dir compare="Text">serial_avg_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_avg_int16">
+        <output-dir compare="Text">serial_avg_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_avg_int16_null">
+        <output-dir compare="Text">serial_avg_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_avg_int32">
+        <output-dir compare="Text">serial_avg_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_avg_int32_null">
+        <output-dir compare="Text">serial_avg_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_avg_int64">
+        <output-dir compare="Text">serial_avg_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_avg_int64_null">
+        <output-dir compare="Text">serial_avg_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_avg_float">
+        <output-dir compare="Text">serial_avg_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_avg_float_null">
+        <output-dir compare="Text">serial_avg_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_avg_double">
+        <output-dir compare="Text">serial_avg_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_avg_double_null">
+        <output-dir compare="Text">serial_avg_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_kurtosis_double">
+        <output-dir compare="Text">serial_kurtosis_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_kurtosis_double_null">
+        <output-dir compare="Text">serial_kurtosis_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_kurtosis_empty">
+        <output-dir compare="Text">serial_kurtosis_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_kurtosis_float">
+        <output-dir compare="Text">serial_kurtosis_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_kurtosis_float_null">
+        <output-dir compare="Text">serial_kurtosis_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_kurtosis_int16">
+        <output-dir compare="Text">serial_kurtosis_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_kurtosis_int16_null">
+        <output-dir compare="Text">serial_kurtosis_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_kurtosis_int32">
+        <output-dir compare="Text">serial_kurtosis_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_kurtosis_int32_null">
+        <output-dir compare="Text">serial_kurtosis_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_kurtosis_int64">
+        <output-dir compare="Text">serial_kurtosis_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_kurtosis_int64_null">
+        <output-dir compare="Text">serial_kurtosis_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_kurtosis_int8">
+        <output-dir compare="Text">serial_kurtosis_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_kurtosis_int8_null">
+        <output-dir compare="Text">serial_kurtosis_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_skewness_double">
+        <output-dir compare="Text">serial_skewness_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_skewness_double_null">
+        <output-dir compare="Text">serial_skewness_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_skewness_empty">
+        <output-dir compare="Text">serial_skewness_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_skewness_float">
+        <output-dir compare="Text">serial_skewness_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_skewness_float_null">
+        <output-dir compare="Text">serial_skewness_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_skewness_int16">
+        <output-dir compare="Text">serial_skewness_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_skewness_int16_null">
+        <output-dir compare="Text">serial_skewness_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_skewness_int32">
+        <output-dir compare="Text">serial_skewness_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_skewness_int32_null">
+        <output-dir compare="Text">serial_skewness_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_skewness_int64">
+        <output-dir compare="Text">serial_skewness_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_skewness_int64_null">
+        <output-dir compare="Text">serial_skewness_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_skewness_int8">
+        <output-dir compare="Text">serial_skewness_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_skewness_int8_null">
+        <output-dir compare="Text">serial_skewness_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_stddev_double">
+        <output-dir compare="Text">serial_stddev_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_stddev_double_null">
+        <output-dir compare="Text">serial_stddev_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_stddev_empty">
+        <output-dir compare="Text">serial_stddev_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_stddev_float">
+        <output-dir compare="Text">serial_stddev_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_stddev_float_null">
+        <output-dir compare="Text">serial_stddev_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_stddev_int16">
+        <output-dir compare="Text">serial_stddev_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_stddev_int16_null">
+        <output-dir compare="Text">serial_stddev_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_stddev_int32">
+        <output-dir compare="Text">serial_stddev_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_stddev_int32_null">
+        <output-dir compare="Text">serial_stddev_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_stddev_int64">
+        <output-dir compare="Text">serial_stddev_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_stddev_int64_null">
+        <output-dir compare="Text">serial_stddev_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_stddev_int8">
+        <output-dir compare="Text">serial_stddev_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_stddev_int8_null">
+        <output-dir compare="Text">serial_stddev_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/serial_sum_double">
+        <output-dir compare="Text">sum/serial_sum_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/serial_sum_double_null">
+        <output-dir compare="Text">sum/serial_sum_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/serial_sum_empty">
+        <output-dir compare="Text">sum/serial_sum_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/serial_sum_float">
+        <output-dir compare="Text">sum/serial_sum_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/serial_sum_float_null">
+        <output-dir compare="Text">sum/serial_sum_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/serial_sum_int16">
+        <output-dir compare="Text">sum/serial_sum_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/serial_sum_int16_null">
+        <output-dir compare="Text">sum/serial_sum_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/serial_sum_int32">
+        <output-dir compare="Text">sum/serial_sum_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/serial_sum_int32_null">
+        <output-dir compare="Text">sum/serial_sum_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/serial_sum_int64">
+        <output-dir compare="Text">sum/serial_sum_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/serial_sum_int64_null">
+        <output-dir compare="Text">sum/serial_sum_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/serial_sum_int64_overflow">
+        <output-dir compare="Text">sum/serial_sum_int64_overflow</output-dir>
+        <expected-error>Overflow in agg-sum</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/serial_sum_int8">
+        <output-dir compare="Text">sum/serial_sum_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/serial_sum_int8_null">
+        <output-dir compare="Text">sum/serial_sum_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_var_double">
+        <output-dir compare="Text">serial_var_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_var_double_null">
+        <output-dir compare="Text">serial_var_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_var_empty">
+        <output-dir compare="Text">serial_var_empty</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_var_float">
+        <output-dir compare="Text">serial_var_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_var_float_null">
+        <output-dir compare="Text">serial_var_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_var_int16">
+        <output-dir compare="Text">serial_var_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_var_int16_null">
+        <output-dir compare="Text">serial_var_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_var_int32">
+        <output-dir compare="Text">serial_var_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_var_int32_null">
+        <output-dir compare="Text">serial_var_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_var_int64">
+        <output-dir compare="Text">serial_var_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_var_int64_null">
+        <output-dir compare="Text">serial_var_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_var_int8">
+        <output-dir compare="Text">serial_var_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="serial_var_int8_null">
+        <output-dir compare="Text">serial_var_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="skewness_double">
+        <output-dir compare="Text">skewness_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="skewness_double_null">
+        <output-dir compare="Text">skewness_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="skewness_empty_01">
+        <output-dir compare="Text">skewness_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="skewness_empty_02">
+        <output-dir compare="Text">skewness_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="skewness_float">
+        <output-dir compare="Text">skewness_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="skewness_float_null">
+        <output-dir compare="Text">skewness_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="skewness_int16">
+        <output-dir compare="Text">skewness_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="skewness_int16_null">
+        <output-dir compare="Text">skewness_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="skewness_int32">
+        <output-dir compare="Text">skewness_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="skewness_int32_null">
+        <output-dir compare="Text">skewness_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="skewness_int64">
+        <output-dir compare="Text">skewness_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="skewness_int64_null">
+        <output-dir compare="Text">skewness_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="skewness_int8">
+        <output-dir compare="Text">skewness_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="skewness_int8_null">
+        <output-dir compare="Text">skewness_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="skewness_distinct">
+        <output-dir compare="Text">skewness_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_double">
+        <output-dir compare="Text">stddev_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_double_null">
+        <output-dir compare="Text">stddev_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_empty_01">
+        <output-dir compare="Text">stddev_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_empty_02">
+        <output-dir compare="Text">stddev_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_float">
+        <output-dir compare="Text">stddev_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_float_null">
+        <output-dir compare="Text">stddev_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_int16">
+        <output-dir compare="Text">stddev_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_int16_null">
+        <output-dir compare="Text">stddev_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_int32">
+        <output-dir compare="Text">stddev_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_int32_null">
+        <output-dir compare="Text">stddev_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_int64">
+        <output-dir compare="Text">stddev_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_int64_null">
+        <output-dir compare="Text">stddev_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_int8">
+        <output-dir compare="Text">stddev_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_int8_null">
+        <output-dir compare="Text">stddev_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_distinct">
+        <output-dir compare="Text">stddev_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_misc">
+        <output-dir compare="Text">stddev_misc</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="stddev_pop_misc">
+        <output-dir compare="Text">stddev_pop_misc</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_double">
+        <output-dir compare="Text">sum/sum_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_double_null">
+        <output-dir compare="Text">sum/sum_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_empty_01">
+        <output-dir compare="Text">sum/sum_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_empty_02">
+        <output-dir compare="Text">sum/sum_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_float">
+        <output-dir compare="Text">sum/sum_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_float_null">
+        <output-dir compare="Text">sum/sum_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_int16">
+        <output-dir compare="Text">sum/sum_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_int16_null">
+        <output-dir compare="Text">sum/sum_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_int32">
+        <output-dir compare="Text">sum/sum_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_int32_null">
+        <output-dir compare="Text">sum/sum_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_int64">
+        <output-dir compare="Text">sum/sum_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_int64_null">
+        <output-dir compare="Text">sum/sum_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_int64_overflow">
+        <output-dir compare="Text">sum/sum_int64_overflow</output-dir>
+        <expected-error>Overflow in agg-sum</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_int8">
+        <output-dir compare="Text">sum/sum_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_int8_null">
+        <output-dir compare="Text">sum/sum_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_null-with-pred">
+        <output-dir compare="Text">sum/sum_null-with-pred</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_numeric_null">
+        <output-dir compare="Text">sum/sum_numeric_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="sum/sum_distinct">
+        <output-dir compare="Text">sum/sum_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_double">
+        <output-dir compare="Text">var_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_double_null">
+        <output-dir compare="Text">var_double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_empty_01">
+        <output-dir compare="Text">var_empty_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_empty_02">
+        <output-dir compare="Text">var_empty_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_float">
+        <output-dir compare="Text">var_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_float_null">
+        <output-dir compare="Text">var_float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_int16">
+        <output-dir compare="Text">var_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_int16_null">
+        <output-dir compare="Text">var_int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_int32">
+        <output-dir compare="Text">var_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_int32_null">
+        <output-dir compare="Text">var_int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_int64">
+        <output-dir compare="Text">var_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_int64_null">
+        <output-dir compare="Text">var_int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_int8">
+        <output-dir compare="Text">var_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_int8_null">
+        <output-dir compare="Text">var_int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_distinct">
+        <output-dir compare="Text">var_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_misc">
+        <output-dir compare="Text">var_misc</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql">
+      <compilation-unit name="var_pop_misc">
+        <output-dir compare="Text">var_pop_misc</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="aggregate-sql-sugar">
+    <test-case FilePath="aggregate-sql-sugar">
+      <compilation-unit name="array_agg">
+        <output-dir compare="Text">array_agg</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql-sugar">
+      <compilation-unit name="array_agg_negative">
+        <output-dir compare="Text">array_agg</output-dir>
+        <expected-error>ASX1079: Compilation error: arrayagg is a SQL-92 aggregate function. The SQL++ core aggregate function strict_arrayagg could potentially express the intent.</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql-sugar" check-warnings="true">
+      <compilation-unit name="avg_mixed">
+        <output-dir compare="Text">avg_mixed</output-dir>
+        <expected-warn>ASX0004: Unsupported type: agg-avg cannot process input type string (in line 26, at column 12)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: agg-avg cannot process input type string (in line 28, at column 19)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: agg-avg cannot process input type string (in line 29, at column 19)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql-sugar">
+      <compilation-unit name="distinct_mixed">
+        <output-dir compare="Text">distinct_mixed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql-sugar">
+      <compilation-unit name="stddev">
+        <output-dir compare="Text">stddev</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-sql-sugar">
+      <compilation-unit name="var">
+        <output-dir compare="Text">var</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="aggregate-subclause">
+    <test-case FilePath="aggregate-subclause">
+      <compilation-unit name="agg_filter_01">
+        <output-dir compare="Text">agg_filter_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate-subclause">
+      <compilation-unit name="agg_filter_02_neg">
+        <output-dir compare="Text">agg_filter_01</output-dir>
+        <expected-error>ASX1121: Illegal use of aggregate FILTER clause</expected-error>
+        <expected-error>ASX1121: Illegal use of aggregate FILTER clause</expected-error>
+        <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier r</expected-error>
+        <expected-error>ASX1121: Illegal use of aggregate FILTER clause</expected-error>
+        <expected-error>ASX1121: Illegal use of aggregate FILTER clause</expected-error>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="array_fun">
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_prepend">
+        <output-dir compare="Text">array_prepend</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_append">
+        <output-dir compare="Text">array_append</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_position">
+        <output-dir compare="Text">array_position</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_repeat">
+        <output-dir compare="Text">array_repeat</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_reverse">
+        <output-dir compare="Text">array_reverse</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_contains">
+        <output-dir compare="Text">array_contains</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_insert">
+        <output-dir compare="Text">array_insert</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_put">
+        <output-dir compare="Text">array_put</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_remove">
+        <output-dir compare="Text">array_remove</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_distinct">
+        <output-dir compare="Text">array_distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_sort">
+        <output-dir compare="Text">array_sort</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_concat">
+        <output-dir compare="Text">array_concat</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_flatten">
+        <output-dir compare="Text">array_flatten</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_ifnull">
+        <output-dir compare="Text">array_ifnull</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_intersect">
+        <output-dir compare="Text">array_intersect</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_range">
+        <output-dir compare="Text">array_range</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_replace">
+        <output-dir compare="Text">array_replace</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_star">
+        <output-dir compare="Text">array_star</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_symdiff">
+        <output-dir compare="Text">array_symdiff</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_symdiffn">
+        <output-dir compare="Text">array_symdiffn</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_union">
+        <output-dir compare="Text">array_union</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_slice/array_slice_double_argument">
+        <output-dir compare="Text">array_slice/array_slice_double_argument</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_slice/array_slice_int_argument">
+        <output-dir compare="Text">array_slice/array_slice_int_argument</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_slice/array_slice_negative_argument">
+        <output-dir compare="Text">array_slice/array_slice_negative_argument</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_slice/array_slice_missing_result">
+        <output-dir compare="Text">array_slice/array_slice_missing_result</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_slice/array_slice_null_result">
+        <output-dir compare="Text">array_slice/array_slice_null_result</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_slice/array_slice_exception_result">
+        <output-dir compare="Text">array_slice/array_slice_exception_result</output-dir>
+        <expected-error>ASX1081: Cannot find function with signature TinySocial.array_slice()</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_except/001">
+        <output-dir compare="Text">array_except/001</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_except/002">
+        <output-dir compare="Text">array_except/002</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_except/003">
+        <output-dir compare="Text">array_except/003</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_except/004">
+        <output-dir compare="Text">array_except/004</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_except/005">
+        <output-dir compare="Text">array_except/005</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_except/006">
+        <output-dir compare="Text">array_except/006</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_errors">
+        <output-dir compare="Text">array_errors</output-dir>
+        <expected-error>Input contains different list types (in line 25, at column 8)</expected-error>
+        <expected-error>Input contains different list types (in line 25, at column 8)</expected-error>
+        <expected-error>Input contains different list types (in line 25, at column 8)</expected-error>
+        <expected-error>Input contains different list types (in line 25, at column 8)</expected-error>
+        <expected-error>Input contains different list types (in line 25, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="bitwise">
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_and_01">
+        <output-dir compare="Text">bit_and_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_and_02">
+        <output-dir compare="Text">bit_and_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_and_03">
+        <output-dir compare="Text">bit_and_03</output-dir>
+        <expected-error>Invalid number of arguments for function</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_and_04">
+        <output-dir compare="Text">bit_and_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_or_01">
+        <output-dir compare="Text">bit_or_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_or_02">
+        <output-dir compare="Text">bit_or_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_or_03">
+        <output-dir compare="Text">bit_or_03</output-dir>
+        <expected-error>Invalid number of arguments for function</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_or_04">
+        <output-dir compare="Text">bit_or_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_xor_01">
+        <output-dir compare="Text">bit_xor_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_xor_02">
+        <output-dir compare="Text">bit_xor_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_xor_03">
+        <output-dir compare="Text">bit_xor_03</output-dir>
+        <expected-error>Invalid number of arguments for function</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_xor_04">
+        <output-dir compare="Text">bit_xor_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_not_01">
+        <output-dir compare="Text">bit_not_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_not_02">
+        <output-dir compare="Text">bit_not_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_not_03">
+        <output-dir compare="Text">bit_not_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_count_01">
+        <output-dir compare="Text">bit_count_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_count_02">
+        <output-dir compare="Text">bit_count_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_count_03">
+        <output-dir compare="Text">bit_count_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_set_01">
+        <output-dir compare="Text">bit_set_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_set_02">
+        <output-dir compare="Text">bit_set_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_set_03">
+        <output-dir compare="Text">bit_set_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_clear_01">
+        <output-dir compare="Text">bit_clear_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_clear_02">
+        <output-dir compare="Text">bit_clear_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_clear_03">
+        <output-dir compare="Text">bit_clear_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_shift_01">
+        <output-dir compare="Text">bit_shift_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_shift_02">
+        <output-dir compare="Text">bit_shift_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_shift_03">
+        <output-dir compare="Text">bit_shift_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_shift_04">
+        <output-dir compare="Text">bit_shift_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_test_01">
+        <output-dir compare="Text">bit_test_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_test_02">
+        <output-dir compare="Text">bit_test_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_test_03">
+        <output-dir compare="Text">bit_test_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="bit_test_04">
+        <output-dir compare="Text">bit_test_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="is_bit_set_01">
+        <output-dir compare="Text">is_bit_set_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="is_bit_set_02">
+        <output-dir compare="Text">is_bit_set_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="bitwise">
+      <compilation-unit name="is_bit_set_03">
+        <output-dir compare="Text">is_bit_set_03</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="boolean">
+    <test-case FilePath="boolean">
+      <compilation-unit name="and_01">
+        <output-dir compare="Text">and_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="boolean">
+      <compilation-unit name="and_null">
+        <output-dir compare="Text">and_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="boolean">
+      <compilation-unit name="and_null_false">
+        <output-dir compare="Text">and_null_false</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="boolean">
+      <compilation-unit name="not_01">
+        <output-dir compare="Text">not_01</output-dir>
+      </compilation-unit>
+    </test-case>
+  </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>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="datetime_order">
+        <output-dir compare="Text">datetime_order</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="datetime_range">
+        <output-dir compare="Text">datetime_range</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="datetime_range_between">
+        <output-dir compare="Text">datetime_range</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="datetime_tzeq">
+        <output-dir compare="Text">datetime_tzeq</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="double">
+        <output-dir compare="Text">double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="double_gte_01">
+        <output-dir compare="Text">double_gte_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="double_missing">
+        <output-dir compare="Text">double_missing</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="double_null">
+        <output-dir compare="Text">double_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="eq_01">
+        <output-dir compare="Text">eq_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="float">
+        <output-dir compare="Text">float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="float_missing">
+        <output-dir compare="Text">float_missing</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="float_null">
+        <output-dir compare="Text">float_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="gt_01">
+        <output-dir compare="Text">gt_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="gte_01">
+        <output-dir compare="Text">gte_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="incompatible">
+        <output-dir compare="Text">incompatible</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="int16">
+        <output-dir compare="Text">int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="int16_missing">
+        <output-dir compare="Text">int16_missing</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="int16_null">
+        <output-dir compare="Text">int16_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="int32">
+        <output-dir compare="Text">int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="int32_missing">
+        <output-dir compare="Text">int32_missing</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="int32_null">
+        <output-dir compare="Text">int32_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="int64">
+        <output-dir compare="Text">int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="int64_missing">
+        <output-dir compare="Text">int64_missing</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="int64_null">
+        <output-dir compare="Text">int64_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="int8">
+        <output-dir compare="Text">int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="int8_missing">
+        <output-dir compare="Text">int8_missing</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="int8_null">
+        <output-dir compare="Text">int8_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="int_not_between">
+        <output-dir compare="Text">int_not_between</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="is_distinct_01">
+        <output-dir compare="Text">is_distinct_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="like">
+        <output-dir compare="Text">like</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="lt_01">
+        <output-dir compare="Text">lt_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="lte_01">
+        <output-dir compare="Text">lte_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="neq_01">
+        <output-dir compare="Text">neq_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="neq_02">
+        <output-dir compare="Text">neq_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="numeric-comparison_01">
+        <output-dir compare="Text">numeric-comparison_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="string">
+        <output-dir compare="Text">string</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="string_missing">
+        <output-dir compare="Text">string_missing</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="string_null">
+        <output-dir compare="Text">string_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="issue363_equality">
+        <output-dir compare="Text">issue363_equality</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="issue363_inequality_duration">
+        <output-dir compare="Text">issue363_inequality_duration</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="issue363_inequality_interval">
+        <output-dir compare="Text">issue363_inequality_interval</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="issue363_inequality_point">
+        <output-dir compare="Text">issue363_inequality_point</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="issue363_inequality_line">
+        <output-dir compare="Text">issue363_inequality_line</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="issue363_inequality_polygon">
+        <output-dir compare="Text">issue363_inequality_polygon</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="issue363_inequality_rectangle">
+        <output-dir compare="Text">issue363_inequality_rectangle</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="issue363_inequality_circle">
+        <output-dir compare="Text">issue363_inequality_circle</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="binary">
+        <output-dir compare="Text">binary</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="binary_null">
+        <output-dir compare="Text">binary_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="greatest_mixed">
+        <output-dir compare="Text">greatest_mixed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="least_mixed">
+        <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-case FilePath="comparison">
+      <compilation-unit name="arrays">
+        <output-dir compare="Text">arrays</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="circle-point">
+        <output-dir compare="Text">circle-point</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison" check-warnings="true">
+      <compilation-unit name="incomparable_types">
+        <output-dir compare="Text">incomparable_types</output-dir>
+          <expected-warn>Incomparable input types: string and bigint (in line 26, at column 13)</expected-warn>
+          <expected-warn>Incomparable input types: array and bigint (in line 23, at column 7)</expected-warn>
+          <expected-warn>Incomparable input types: point and point (in line 24, at column 18)</expected-warn>
+          <expected-warn>Incomparable input types: bigint and string (in line 25, at column 46)</expected-warn>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="constructor">
+    <test-case FilePath="constructor">
+      <compilation-unit name="binary_01">
+        <output-dir compare="Text">binary_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor" check-warnings="true">
+      <compilation-unit name="binary_02">
+        <output-dir compare="Text">binary_02</output-dir>
+        <expected-warn>ASX0006: Invalid format for binary in @#!1 (in line 24, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: hex() cannot process input type date (in line 25, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: hex() cannot process input type array (in line 26, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: hex() cannot process input type object (in line 27, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for binary in @#!2 (in line 24, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: base64() cannot process input type date (in line 25, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: base64() cannot process input type array (in line 26, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: base64() cannot process input type object (in line 27, at column 13)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="add-null">
+        <output-dir compare="Text">add-null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="boolean_01">
+        <output-dir compare="Text">boolean_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor" check-warnings="true">
+      <compilation-unit name="boolean_02">
+        <output-dir compare="Text">boolean_02</output-dir>
+        <expected-warn>ASX0006: Invalid format for boolean in FALSE (in line 24, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for boolean in TRUE (in line 25, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: boolean() cannot process input type date (in line 26, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: boolean() cannot process input type array (in line 27, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: boolean() cannot process input type object (in line 28, at column 13)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="circle_01">
+        <output-dir compare="Text">circle_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="date_01">
+        <output-dir compare="Text">date_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor" check-warnings="true">
+      <compilation-unit name="date_02">
+        <output-dir compare="Text">date_02</output-dir>
+        <expected-warn>ASX0006: Invalid format for date in @#! (in line 24, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: date() cannot process input type boolean (in line 25, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: date() cannot process input type bigint (in line 26, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: date() cannot process input type time (in line 27, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: date() cannot process input type array (in line 28, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: date() cannot process input type object (in line 29, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for date in 12/31/2020 (in line 30, at column 13)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="datetime_01">
+        <output-dir compare="Text">datetime_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor" check-warnings="true">
+      <compilation-unit name="datetime_02">
+        <output-dir compare="Text">datetime_02</output-dir>
+        <expected-warn>ASX0006: Invalid format for datetime in @#! (in line 24, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type boolean (in line 25, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type bigint (in line 26, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type array (in line 27, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type object (in line 28, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for datetime in 1951-12-27T12:20:15Z (in line 29, at column 13)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="double_01">
+        <output-dir compare="Text">double_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor" check-warnings="true">
+      <compilation-unit name="double_02">
+        <output-dir compare="Text">double_02</output-dir>
+        <expected-warn>ASX0006: Invalid format for double in @#! (in line 24, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: double() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: double() cannot process input type date (in line 26, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: double() cannot process input type time (in line 27, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: double() cannot process input type array (in line 28, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: double() cannot process input type object (in line 29, at column 13)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="duration_01">
+        <output-dir compare="Text">duration_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor" check-warnings="true">
+      <compilation-unit name="duration_02">
+        <output-dir compare="Text">duration_02</output-dir>
+        <expected-warn>ASX0006: Invalid format for duration in @#! (in line 24, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: duration() cannot process input type bigint (in line 25, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: duration() cannot process input type datetime (in line 26, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: duration() cannot process input type date (in line 27, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: duration() cannot process input type time (in line 28, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: duration() cannot process input type array (in line 29, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: duration() cannot process input type object (in line 30, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for yearmonthduration in @#! (in line 24, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type bigint (in line 25, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type datetime (in line 26, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type date (in line 27, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type time (in line 28, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type array (in line 29, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type object (in line 30, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for daytimeduration in @#! (in line 24, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type bigint (in line 25, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type datetime (in line 26, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type date (in line 27, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type time (in line 28, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type array (in line 29, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type object (in line 30, at column 13)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="float_01">
+        <output-dir compare="Text">float_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor" check-warnings="true">
+      <compilation-unit name="float_02">
+        <output-dir compare="Text">float_02</output-dir>
+        <expected-warn>ASX0006: Invalid format for float in @#! (in line 24, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: float() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: float() cannot process input type date (in line 26, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: float() cannot process input type time (in line 27, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: float() cannot process input type array (in line 28, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: float() cannot process input type object (in line 29, at column 13)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="int_01">
+        <output-dir compare="Text">int_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor" check-warnings="true">
+      <compilation-unit name="int_02">
+        <output-dir compare="Text">int_02</output-dir>
+        <expected-warn>ASX0006: Invalid format for tinyint in @#! (in line 24, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int8() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int8() cannot process input type date (in line 26, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int8() cannot process input type time (in line 27, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int8() cannot process input type array (in line 28, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int8() cannot process input type object (in line 29, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for smallint in @#! (in line 24, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int16() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int16() cannot process input type date (in line 26, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int16() cannot process input type time (in line 27, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int16() cannot process input type array (in line 28, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int16() cannot process input type object (in line 29, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for integer in @#! (in line 24, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int32() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int32() cannot process input type date (in line 26, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int32() cannot process input type time (in line 27, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int32() cannot process input type array (in line 28, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int32() cannot process input type object (in line 29, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for bigint in @#! (in line 24, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int64() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int64() cannot process input type date (in line 26, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int64() cannot process input type time (in line 27, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int64() cannot process input type array (in line 28, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int64() cannot process input type object (in line 29, at column 13)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="interval">
+        <output-dir compare="Text">interval</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="line_01">
+        <output-dir compare="Text">line_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="rectangle_01">
+        <output-dir compare="Text">rectangle_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="point_01">
+        <output-dir compare="Text">point_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="polygon_01">
+        <output-dir compare="Text">polygon_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="primitive-01">
+        <output-dir compare="Text">primitive-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="primitive-02">
+        <output-dir compare="Text">primitive-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="primitive-03">
+        <output-dir compare="Text">primitive-03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="primitive-04">
+        <output-dir compare="Text">primitive-04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="string_01">
+        <output-dir compare="Text">string_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor" check-warnings="true">
+      <compilation-unit name="string_02">
+        <output-dir compare="Text">string_02</output-dir>
+        <expected-warn>ASX0004: Unsupported type: string() cannot process input type array (in line 24, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: string() cannot process input type object (in line 25, at column 13)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="time_01">
+        <output-dir compare="Text">time_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor" check-warnings="true">
+      <compilation-unit name="time_02">
+        <output-dir compare="Text">time_02</output-dir>
+        <expected-warn>ASX0006: Invalid format for time in @#! (in line 24, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: time() cannot process input type boolean (in line 25, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: time() cannot process input type bigint (in line 26, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: time() cannot process input type date (in line 27, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: time() cannot process input type array (in line 28, at column 13)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: time() cannot process input type object (in line 29, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for time in 11:58:59 (in line 30, at column 13)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor/uuid">
+      <compilation-unit name="uuid_01">
+        <output-dir compare="Text">uuid_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor/uuid" check-warnings="true">
+      <compilation-unit name="uuid_02">
+        <output-dir compare="Text">uuid_02</output-dir>
+        <expected-warn>ASX0006: Invalid format for uuid in 02a199ca-bf58-412e-bd9f-60a0c975a8a- (in line 24, at column 13)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for uuid in 12345 (in line 25, at column 13)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
+      <compilation-unit name="polygon-from-open-list_issue1627">
+        <output-dir compare="Text">polygon-from-open-list_issue1627</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="custord">
+    <!--
+    <test-case FilePath="custord">
+      <compilation-unit name="co">
+        <output-dir compare="Text">co</output-dir>
+      </compilation-unit>
+    </test-case>
+    -->
+    <test-case FilePath="custord">
+      <compilation-unit name="customer_q_01">
+        <output-dir compare="Text">customer_q_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="customer_q_02">
+        <output-dir compare="Text">customer_q_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="customer_q_03">
+        <output-dir compare="Text">customer_q_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="customer_q_04">
+        <output-dir compare="Text">customer_q_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="customer_q_05">
+        <output-dir compare="Text">customer_q_05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="customer_q_06">
+        <output-dir compare="Text">customer_q_06</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="customer_q_07">
+        <output-dir compare="Text">customer_q_07</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="customer_q_08">
+        <output-dir compare="Text">customer_q_08</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--
+        <test-case FilePath="custord">
+          <compilation-unit name="denorm-cust-order_01">
+            <output-dir compare="Text">denorm-cust-order_01</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <test-case FilePath="custord">
+      <compilation-unit name="denorm-cust-order_02">
+        <output-dir compare="Text">denorm-cust-order_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--
+        <test-case FilePath="custord">
+          <compilation-unit name="denorm-cust-order_03">
+            <output-dir compare="Text">denorm-cust-order_03</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <!--
+        <test-case FilePath="custord">
+          <compilation-unit name="freq-clerk">
+            <output-dir compare="Text">freq-clerk</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <test-case FilePath="custord">
+      <compilation-unit name="join_q_01">
+        <output-dir compare="Text">join_q_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="join_q_02">
+        <output-dir compare="Text">join_q_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="join_q_03">
+        <output-dir compare="Text">join_q_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="join_q_04">
+        <output-dir compare="Text">join_q_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="join_q_05">
+        <output-dir compare="Text">join_q_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="join_q_06">
+        <output-dir compare="Text">join_q_06</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="join_q_07">
+        <output-dir compare="Text">join_q_06</output-dir>
+        <expected-error>Cannot find dataset c in dataverse test nor an alias with name c</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="join_q_08">
+        <output-dir compare="Text">join_q_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="join_q_09">
+        <output-dir compare="Text">join_q_01</output-dir>
+        <expected-error>Cannot resolve ambiguous alias reference for identifier age</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="query-ASTERIXDB-1754">
+        <output-dir compare="Text">query-ASTERIXDB-1754</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="load-test">
+        <output-dir compare="Text">load-test</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="order_q_01">
+        <output-dir compare="Text">order_q_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="order_q_02">
+        <output-dir compare="Text">order_q_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="order_q_03">
+        <output-dir compare="Text">order_q_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="order_q_04">
+        <output-dir compare="Text">order_q_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="order_q_05">
+        <output-dir compare="Text">order_q_05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="custord">
+      <compilation-unit name="order_q_06">
+        <output-dir compare="Text">order_q_06</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="dapd">
+    <test-case FilePath="dapd">
+      <compilation-unit name="q1">
+        <output-dir compare="Text">q1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dapd">
+      <compilation-unit name="q2">
+        <output-dir compare="Text">q2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dapd">
+      <compilation-unit name="q2-2">
+        <output-dir compare="Text">q2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dapd">
+      <compilation-unit name="q2-2-negative">
+        <output-dir compare="Text">q2</output-dir>
+        <expected-error>Cannot find dataset e in dataverse test nor an alias with name e</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dapd">
+      <compilation-unit name="q2-3">
+        <output-dir compare="Text">q2</output-dir>
+        <expected-error>Cannot resolve ambiguous alias reference for identifier sig_id</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dapd">
+      <compilation-unit name="q2-4">
+        <output-dir compare="Text">q2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dapd">
+      <compilation-unit name="q2-5">
+        <output-dir compare="Text">q2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dapd">
+      <compilation-unit name="q2-6">
+        <output-dir compare="Text">q2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dapd">
+      <compilation-unit name="q2-7">
+        <output-dir compare="Text">q2-7</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dapd">
+      <compilation-unit name="q2-8">
+        <output-dir compare="Text">q2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dapd">
+      <compilation-unit name="q2-9">
+        <output-dir compare="Text">q2-9</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dapd">
+      <compilation-unit name="q2-10">
+        <output-dir compare="Text">q2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dapd">
+      <compilation-unit name="q2-11">
+        <output-dir compare="Text">q2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dapd">
+      <compilation-unit name="q2-12">
+        <output-dir compare="Text">q2-12</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--
+        <test-case FilePath="dapd">
+          <compilation-unit name="q3">
+            <output-dir compare="Text">q3</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+  </test-group>
+  <test-group name="ddl">
+    <test-case FilePath="ddl/create-index">
+      <compilation-unit name="create-index-1">
+        <output-dir compare="Text">create-index-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl/create-index">
+      <compilation-unit name="create-index-2">
+        <output-dir compare="Text">create-index-2</output-dir>
+        <expected-error>Syntax error: In line 53 >>create  primary index sec_primary_idx1  on LineItem type rtree;&lt;&lt; Encountered "rtree" at column 58.</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl/create-index">
+      <compilation-unit name="create-index-3">
+        <output-dir compare="Text">create-index-3</output-dir>
+        <expected-error>Syntax error: In line 53 >>create  primary  sec_primary_idx1  on LineItem;&lt;&lt; Encountered &lt;IDENTIFIER&gt; "sec_primary_idx1" at column 18.</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl/create-index">
+      <compilation-unit name="create-index-4">
+        <output-dir compare="Text">create-index-4</output-dir>
+        <expected-error>Syntax error: In line 53 >>create  primary index if not exists sec_primary_idx1  if not exists on LineItem;&lt;&lt; Encountered &lt;IDENTIFIER&gt; "sec_primary_idx1" at column 37.</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl/create-index">
+      <compilation-unit name="create-index-5">
+        <output-dir compare="Text">create-index-5</output-dir>
+        <expected-error>Syntax error: In line 53 >>create  primary index if exists sec_primary_idx1  on LineItem;&lt;&lt; Encountered "exists" at column 26.</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl/create-index">
+      <compilation-unit name="create-index-6">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1050: Cannot find dataset with name LineItemView1 in dataverse test (in line 55, at column 1)</expected-error>
+        <expected-error>ASX1050: Cannot find dataset with name LineItemView2 in dataverse test (in line 60, at column 1)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl/create-index">
+      <compilation-unit name="create-inverted-index-with-variable-length-primary-key">
+        <output-dir compare="Text">create-inverted-index-with-variable-length-primary-key</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="bad-type-ddl">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1079: Compilation error: Reserved type name $x</expected-error>
+        <expected-error>ASX0013: Duplicate field name 'c' (in line 29, at column 19)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="create-dataset-inline-type-1">
+        <output-dir compare="Text">create-dataset-inline-type-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="create-dataset-inline-type-2">
+        <output-dir compare="Text">create-dataset-inline-type-2</output-dir>
+        <expected-error>ASX1082: Cannot find datatype with name test.$d$t$i$Cust1</expected-error>
+        <expected-error>ASX1082: Cannot find datatype with name test.$d$t$i$Cust2</expected-error>
+        <expected-error>ASX1082: Cannot find datatype with name my_unknown_type</expected-error>
+        <expected-error>ASX0013: Duplicate field name 'c_name' (in line 25, at column 22)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="drop-primary-index">
+        <output-dir compare="Text">drop-primary-index</output-dir>
+        <expected-error>Cannot drop index 'ds'. Drop dataset 'ds' to remove this index</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl" check-warnings="true">
+      <compilation-unit name="invalid-dataverse">
+        <output-dir compare="Text">invalid-dataverse</output-dir>
+        <source-location>false</source-location>
+        <expected-warn>Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-warn>
+        <expected-error>Cannot find dataverse with name fakeDataverse (in line 27, at column 1)</expected-error>
+        <expected-warn>Cannot find dataverse with name fakeDataverse (in line 29, at column 1)</expected-warn>
+        <expected-error>Cannot find dataverse with name fakeDataverse (in line 30, at column 1)</expected-error>
+        <expected-error>Cannot find datatype with name fakeDataverse.myType</expected-error>
+        <expected-error>Cannot find dataverse with name fakeDataverse (in line 30, at column 1)</expected-error>
+        <expected-error>Cannot find dataverse with name fakeDataverse (in line 32, at column 1)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="invalid-dataverse-name">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
+        <expected-error>ASX1079: Compilation error: Invalid operation - Cannot create dataverse: asterix</expected-error>
+        <expected-error>ASX1079: Compilation error: Invalid operation - Cannot create dataverse: algebricks</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: '' (in line 24, at column 16)</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 27, at column 16)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="invalid-dataset-name">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 16)</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 31, at column 14)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="invalid-feed-name">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 34, at column 13)</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 42, at column 11)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="invalid-feed-policy-name">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 1)</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 32, at column 23)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="invalid-index-name">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 19)</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 33, at column 12)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="invalid-nodegroup-name">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="invalid-type-name">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 13)</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 33, at column 11)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="invalid-udf-name">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1115: Invalid name for a database object: ''</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: ' a'</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: ' invalid'</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 17)</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 33, at column 15)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="invalid-view-name">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1115: Invalid name for a database object: '' (in line 29, at column 1)</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: ' a' (in line 29, at column 1)</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: ' invalid' (in line 29, at column 1)</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: 'a/b' (in line 29, at column 13)</expected-error>
+        <expected-error>ASX1115: Invalid name for a database object: 'c/d' (in line 32, at column 11)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="dataset-and-index-same-dataverse">
+        <output-dir compare="Text">dataset-and-index-same-dataverse</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl" check-warnings="true">
+      <compilation-unit name="drop_dataset_invalid_dataverse">
+        <output-dir compare="Text">drop_dataset_invalid_dataverse</output-dir>
+        <expected-error>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-error>
+        <expected-error>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-error>
+        <expected-error>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-error>
+        <expected-error>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-error>
+        <expected-error>ASX1050: Cannot find dataset with name fakeDataset1 in dataverse realDataverse (in line 22, at column 1)</expected-error>
+        <expected-warn>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-warn>
+        <expected-warn>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-warn>
+        <expected-warn>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="create_dataset_with_filter_on_meta">
+        <output-dir compare="Text">create_dataset_with_filter_on_meta</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="index-bad-fields">
+        <output-dir compare="Text">index-bad-fields</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="index-cast-null">
+        <placeholder name="with" value="" />
+        <output-dir compare="Text">index-cast-null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="index-cast-null">
+        <placeholder name="with" value="WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" />
+        <output-dir compare="Text">index-cast-null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl">
+      <compilation-unit name="index-cast-null-negative">
+        <output-dir compare="Text">index-cast-null-negative</output-dir>
+        <expected-error>CAST modifier is only allowed for B-Tree indexes</expected-error>
+        <expected-error>CAST modifier cannot be specified together with ENFORCED</expected-error>
+        <expected-error>CAST modifier is used without specifying the type of the indexed field</expected-error>
+        <expected-error>Typed index on 'typed_f2' field could be created only for open datatype</expected-error>
+        <expected-error>Parameter invalid_date cannot be set</expected-error>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="dml">
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-with-autogenerated-pk_adm-with-sec-primary-index">
+        <output-dir compare="Text">insert-with-autogenerated-pk_adm-with-sec-primary-index</output-dir>
+      </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="compact-dataset-and-its-indexes">
+        <output-dir compare="Text">compact-dataset-and-its-indexes</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-constant-merge-policy">
+        <output-dir compare="Text">using-constant-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-prefix-merge-policy">
+        <output-dir compare="Text">using-prefix-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-concurrent-merge-policy">
+        <output-dir compare="Text">using-concurrent-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-correlated-prefix-merge-policy">
+        <output-dir compare="Text">using-correlated-prefix-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-correlated-prefix-merge-policy-with-feed">
+        <output-dir compare="Text">using-correlated-prefix-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="using-no-merge-policy">
+        <output-dir compare="Text">using-no-merge-policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="query-issue382">
+        <output-dir compare="Text">query-issue382</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="query-issue433">
+        <output-dir compare="Text">query-issue433</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="query-issue288">
+        <output-dir compare="Text">query-issue288</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="query-issue205">
+        <output-dir compare="Text">query-issue205</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="delete-from-loaded-dataset-with-index">
+        <output-dir compare="Text">delete-from-loaded-dataset-with-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="delete-from-loaded-dataset">
+        <output-dir compare="Text">delete-from-loaded-dataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="delete-syntax-change">
+        <output-dir compare="Text">delete-syntax-change</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="drop-empty-secondary-indexes">
+        <output-dir compare="Text">drop-empty-secondary-indexes</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="drop-index">
+        <output-dir compare="Text">drop-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="create-drop-cltype">
+        <output-dir compare="Text">create-drop-cltype</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="create-drop-opntype">
+        <output-dir compare="Text">create-drop-opntype</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="empty-load-with-index">
+        <output-dir compare="Text">empty-load-with-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-into-empty-dataset">
+        <output-dir compare="Text">insert-into-empty-dataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--test-case FilePath="dml">
+      <compilation-unit name="insert-into-empty-dataset-with-index">
+        <output-dir compare="Text">insert-into-empty-dataset-with-index</output-dir>
+      </compilation-unit>
+    </test-case!-->
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-syntax">
+        <output-dir compare="Text">insert-syntax</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-and-scan-dataset">
+        <output-dir compare="Text">insert-and-scan-dataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-and-scan-dataset-with-index">
+        <output-dir compare="Text">insert-and-scan-dataset-with-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="recreate-index">
+        <output-dir compare="Text">recreate-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-and-scan-joined-datasets">
+        <output-dir compare="Text">insert-and-scan-joined-datasets</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--test-case FilePath="dml">
+      <compilation-unit name="insert-into-loaded-dataset-with-index_01">
+        <output-dir compare="Text">insert-into-loaded-dataset-with-index_01</output-dir>
+      </compilation-unit>
+    </test-case!-->
+    <!--test-case FilePath="dml">
+      <compilation-unit name="insert-into-loaded-dataset-with-index_02">
+        <output-dir compare="Text">insert-into-loaded-dataset-with-index_02</output-dir>
+      </compilation-unit>
+    </test-case!-->
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-into-loaded-dataset_01">
+        <output-dir compare="Text">insert-into-loaded-dataset_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-into-loaded-dataset_02">
+        <output-dir compare="Text">insert-into-loaded-dataset_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-src-dst-01">
+        <output-dir compare="Text">insert-src-dst-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert">
+        <output-dir compare="Text">insert</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <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">
+      <compilation-unit name="insert-duplicated-keys-with-pk-index">
+        <output-dir compare="Text">insert-duplicated-keys-with-pk-index</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">
+      <compilation-unit name="insert_less_nc">
+        <output-dir compare="Text">insert_less_nc</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--
+        <test-case FilePath="dml">
+          <compilation-unit name="load-from-hdfs">
+            <output-dir compare="Text">load-from-hdfs</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-with-autogenerated-pk_adm-with-sec-primary-index">
+        <output-dir compare="Text">insert-with-autogenerated-pk_adm-with-sec-primary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-with-autogenerated-pk_adm_01">
+        <output-dir compare="Text">insert-with-autogenerated-pk_adm_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <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>Field type string cannot be promoted to type uuid</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-with-autogenerated-pk_adm_03">
+        <output-dir compare="Text">insert-with-autogenerated-pk_adm_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="load-with-autogenerated-pk_txt_01">
+        <output-dir compare="Text">load-with-autogenerated-pk_txt_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="load-with-autogenerated-pk_adm_01">
+        <output-dir compare="Text">load-with-autogenerated-pk_adm_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <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</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</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="load-with-autogenerated-pk_adm_04">
+        <output-dir compare="Text">load-with-autogenerated-pk_adm_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="load-with-autogenerated-pk_csv_01">
+        <output-dir compare="Text">load-with-autogenerated-pk_csv_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="load-with-autogenerated-pk_csv_02">
+        <output-dir compare="Text">load-with-autogenerated-pk_csv_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <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</expected-error>
+        <expected-error>ASX1014: Field 'not_id' is not found</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-return-records">
+        <output-dir compare="Text">insert-return-records</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-returning-udf">
+        <output-dir compare="Text">insert-returning-fieldname</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-returning-fieldname">
+        <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>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-returning-fieldname-implicit-2">
+        <output-dir compare="Text">insert-returning-fieldname</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert_uuid_autogenerate">
+        <output-dir compare="Text">insert_uuid_autogenerate</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="upsert_uuid_autogenerate">
+        <output-dir compare="Text">upsert_uuid_autogenerate</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert_uuid_manual">
+        <output-dir compare="Text">insert_uuid_manual</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="upsert_uuid_manual">
+        <output-dir compare="Text">upsert_uuid_manual</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert_uuid_manual_exists">
+        <output-dir compare="Text">insert_uuid_manual_exists</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">
+      <compilation-unit name="upsert_uuid_manual_exists">
+        <output-dir compare="Text">upsert_uuid_manual_exists</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert_uuid_manual_exists_select">
+        <output-dir compare="Text">insert_uuid_manual_exists_select</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">
+      <compilation-unit name="upsert_uuid_manual_exists_select">
+        <output-dir compare="Text">upsert_uuid_manual_exists_select</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert_nested_uuid_autogenerate">
+        <output-dir compare="Text">insert_nested_uuid_autogenerate</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="upsert_nested_uuid_autogenerate">
+        <output-dir compare="Text">upsert_nested_uuid_autogenerate</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert_nested_uuid_manual">
+        <output-dir compare="Text">insert_nested_uuid_manual</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="upsert_nested_uuid_manual">
+        <output-dir compare="Text">upsert_nested_uuid_manual</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert_nested_uuid_manual_exists">
+        <output-dir compare="Text">insert_nested_uuid_manual_exists</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">
+      <compilation-unit name="upsert_nested_uuid_manual_exists">
+        <output-dir compare="Text">upsert_nested_uuid_manual_exists</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert_nested_uuid_manual_exists_select">
+        <output-dir compare="Text">insert_nested_uuid_manual_exists_select</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">
+      <compilation-unit name="upsert_nested_uuid_manual_exists_select">
+        <output-dir compare="Text">upsert_nested_uuid_manual_exists_select</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-with-bad-return">
+        <output-dir compare="Text">insert-with-bad-return</output-dir>
+        <expected-error>A returning expression cannot contain dataset access</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="load-with-index">
+        <output-dir compare="Text">load-with-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="load-with-ngram-index">
+        <output-dir compare="Text">load-with-ngram-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="load-with-rtree-index">
+        <output-dir compare="Text">load-with-rtree-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="load-with-word-index">
+        <output-dir compare="Text">load-with-word-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="opentype-c2o-recursive">
+        <output-dir compare="Text">opentype-c2o-recursive</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="opentype-c2o">
+        <output-dir compare="Text">opentype-c2o</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="opentype-closed-optional">
+        <output-dir compare="Text">opentype-closed-optional</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="opentype-insert">
+        <output-dir compare="Text">opentype-insert</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="opentype-insert2">
+        <output-dir compare="Text">opentype-insert2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="opentype-noexpand">
+        <output-dir compare="Text">opentype-noexpand</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="opentype-o2c-recursive">
+        <output-dir compare="Text">opentype-o2c-recursive</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="opentype-o2c">
+        <output-dir compare="Text">opentype-o2c</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="opentype-o2o">
+        <output-dir compare="Text">opentype-o2o</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-btree-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit
+              name="scan-delete-btree-correlated-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-rtree-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-rtree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-rtree-correlated-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-rtree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-rtree-secondary-index">
+        <output-dir compare="Text">scan-delete-rtree-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-rtree-correlated-secondary-index">
+        <output-dir compare="Text">scan-delete-rtree-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-btree-secondary-index-nullable">
+        <output-dir compare="Text">scan-insert-btree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-btree-correlated-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-rtree-secondary-index-nullable">
+        <output-dir compare="Text">scan-insert-rtree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-rtree-correlated-secondary-index-nullable">
+        <output-dir compare="Text">scan-insert-rtree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-rtree-secondary-index">
+        <output-dir compare="Text">scan-insert-rtree-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-rtree-correlated-secondary-index">
+        <output-dir compare="Text">scan-insert-rtree-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index">
+        <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index-string-as-primary-key">
+        <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-string-as-primary-key</output-dir>
+      </compilation-unit>
+    </test-case>
+
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-ngram-correlated-secondary-index">
+        <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-word-secondary-index">
+        <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-word-secondary-index-string-as-primary-key">
+        <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-string-as-primary-key</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-word-correlated-secondary-index">
+        <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-word-correlated-secondary-index-string-as-primary-key">
+        <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-string-as-primary-key</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index-nullable">
+        <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index-nullable-string-as-primary-key">
+        <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-nullable-string-as-primary-key</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-ngram-correlated-secondary-index-nullable">
+        <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-ngram-correlated-secondary-index-nullable-string-as-primary-key">
+        <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-nullable-string-as-primary-key</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-word-secondary-index-nullable">
+        <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-word-secondary-index-nullable-string-as-primary-key">
+        <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-nullable-string-as-primary-key</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-word-correlated-secondary-index-nullable">
+        <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index">
+        <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index-string-as-primary-key">
+        <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-string-as-primary-key</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-ngram-correlated-secondary-index">
+        <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-word-secondary-index">
+        <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-word-correlated-secondary-index">
+        <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-ngram-correlated-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-word-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-word-correlated-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="load-with-index-open">
+        <output-dir compare="Text">load-with-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="load-with-ngram-index-open">
+        <output-dir compare="Text">load-with-ngram-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="load-with-rtree-index-open">
+        <output-dir compare="Text">load-with-rtree-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="load-with-word-index-open">
+        <output-dir compare="Text">load-with-word-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-btree-secondary-index-open">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-btree-correlated-secondary-index-open">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index-open">
+        <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-ngram-correlated-secondary-index-open">
+        <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-word-secondary-index-open">
+        <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-word-correlated-secondary-index-open">
+        <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-rtree-secondary-index-open">
+        <output-dir compare="Text">scan-delete-rtree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-rtree-correlated-secondary-index-open">
+        <output-dir compare="Text">scan-delete-rtree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-btree-secondary-index-open">
+        <output-dir compare="Text">scan-insert-btree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-btree-correlated-secondary-index-open">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index-open">
+        <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-ngram-correlated-secondary-index-open">
+        <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-word-secondary-index-open">
+        <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-word-correlated-secondary-index-open">
+        <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-rtree-secondary-index-open">
+        <output-dir compare="Text">scan-insert-rtree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-rtree-correlated-secondary-index-open">
+        <output-dir compare="Text">scan-insert-rtree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="delete-multi-statement">
+        <output-dir compare="Text">delete-multi-statement</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="upsert-dataset-with-meta">
+        <output-dir compare="Text">upsert-dataset-with-meta</output-dir>
+        <expected-error>upsert into dataset is not supported on datasets with meta records</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="upsert-return-custom-result">
+        <output-dir compare="Text">upsert-return-custom-result</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="upsert-returning-fieldname">
+        <output-dir compare="Text">insert-returning-fieldname</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="upsert-returning-fieldname-implicit">
+        <output-dir compare="Text">insert-returning-fieldname</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="upsert-returning-fieldname-implicit-2">
+        <output-dir compare="Text">insert-returning-fieldname</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="delete-dataset-with-meta">
+        <output-dir compare="Text">delete-dataset-with-meta</output-dir>
+        <expected-error>delete from dataset is not supported on datasets with meta records</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-dataset-with-meta">
+        <output-dir compare="Text">insert-dataset-with-meta</output-dir>
+        <expected-error>insert into dataset is not supported on datasets with meta records</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-duplicated-keys-from-query">
+        <output-dir compare="Text">insert-duplicated-keys-from-query</output-dir>
+        <expected-error>HYR0033: Inserting duplicate keys into the primary storage</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <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</expected-error>
+        <expected-error>ASX1014: Field 'not_id' is not found</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="load-with-index-open_02">
+        <output-dir compare="Text">load-with-index-open_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="query-ASTERIXDB-867">
+        <output-dir compare="Text">query-ASTERIXDB-867</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="query-ASTERIXDB-1406">
+        <output-dir compare="Text">query-ASTERIXDB-1406</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="query-issue382">
+        <output-dir compare="Text">query-issue382</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="create-index-unknown-key">
+        <output-dir compare="Text">index-unknown-key</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-into-index-unknown-key">
+        <output-dir compare="Text">index-unknown-key</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="upsert-into-index-unknown-key">
+        <output-dir compare="Text">index-unknown-key</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="load-into-index-unknown-key">
+        <output-dir compare="Text">index-unknown-key</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="employee">
+    <test-case FilePath="employee">
+      <compilation-unit name="q_01">
+        <output-dir compare="Text">q_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="employee">
+      <compilation-unit name="q_02">
+        <output-dir compare="Text">q_02</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="failure">
+    <test-case FilePath="failure">
+      <compilation-unit name="group_by_failure">
+        <output-dir compare="Text">group_by_failure</output-dir>
+        <expected-error>Injected failure in inject-failure</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="failure">
+      <compilation-unit name="group_by_hash_failure">
+        <output-dir compare="Text">group_by_hash_failure</output-dir>
+        <expected-error>Injected failure in inject-failure</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="failure">
+      <compilation-unit name="q01_pricing_summary_report_failure">
+        <output-dir compare="Text">q01_pricing_summary_report_failure</output-dir>
+        <expected-error>Injected failure in inject-failure</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="failure">
+      <compilation-unit name="q18_large_volume_customer_failure">
+        <output-dir compare="Text">q18_large_volume_customer_failure</output-dir>
+        <expected-error>Injected failure in inject-failure</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="failure">
+      <compilation-unit name="order_by_failure">
+        <output-dir compare="Text">order_by_failure</output-dir>
+        <expected-error>Injected failure in inject-failure</expected-error>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="fuzzyjoin">
+    <test-case FilePath="fuzzyjoin">
+      <compilation-unit name="basic-1_1">
+        <output-dir compare="Text">basic-1_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fuzzyjoin">
+      <compilation-unit name="basic-1_1_2">
+        <output-dir compare="Text">basic-1_1_2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fuzzyjoin">
+      <compilation-unit name="basic-1_1_3">
+        <output-dir compare="Text">basic-1_1_3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fuzzyjoin">
+      <compilation-unit name="dblp-string-as-primary-key">
+        <output-dir compare="Text">dblp-string-as-primary-key</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <!--
+    <test-group name="flwor">
+      <test-case FilePath="flwor">
+        <compilation-unit name="for01">
+          <output-dir compare="Text">for01</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for02">
+          <output-dir compare="Text">for02</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for03">
+          <output-dir compare="Text">for03</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for04">
+          <output-dir compare="Text">for04</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for05">
+          <output-dir compare="Text">for05</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for06">
+          <output-dir compare="Text">for06</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for07">
+          <output-dir compare="Text">for07</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for08">
+          <output-dir compare="Text">for08</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for09">
+          <output-dir compare="Text">for09</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for10">
+          <output-dir compare="Text">for10</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for11">
+          <output-dir compare="Text">for11</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for12">
+          <output-dir compare="Text">for12</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for13">
+          <output-dir compare="Text">for13</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for14">
+          <output-dir compare="Text">for14</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for15">
+          <output-dir compare="Text">for15</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for16">
+          <output-dir compare="Text">for16</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for17">
+          <output-dir compare="Text">for17</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for18">
+          <output-dir compare="Text">for18</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="for19">
+          <output-dir compare="Text">for19</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="grpby01">
+          <output-dir compare="Text">grpby01</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="grpby02">
+          <output-dir compare="Text">grpby02</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let01">
+          <output-dir compare="Text">let01</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let02">
+          <output-dir compare="Text">let02</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let03">
+          <output-dir compare="Text">let03</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let04">
+          <output-dir compare="Text">let04</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let05">
+          <output-dir compare="Text">let05</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let06">
+          <output-dir compare="Text">let06</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let07">
+          <output-dir compare="Text">let07</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let08">
+          <output-dir compare="Text">let08</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let09">
+          <output-dir compare="Text">let09</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let10">
+          <output-dir compare="Text">let10</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let11">
+          <output-dir compare="Text">let11</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let12">
+          <output-dir compare="Text">let12</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let13">
+          <output-dir compare="Text">let13</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let14">
+          <output-dir compare="Text">let14</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let15">
+          <output-dir compare="Text">let15</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let16">
+          <output-dir compare="Text">let16</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let17">
+          <output-dir compare="Text">let17</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let18">
+          <output-dir compare="Text">let18</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let19">
+          <output-dir compare="Text">let19</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let20">
+          <output-dir compare="Text">let20</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let21">
+          <output-dir compare="Text">let21</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let22">
+          <output-dir compare="Text">let22</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let23">
+          <output-dir compare="Text">let23</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let24">
+          <output-dir compare="Text">let24</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let25">
+          <output-dir compare="Text">let25</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let26">
+          <output-dir compare="Text">let26</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let27">
+          <output-dir compare="Text">let27</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let28">
+          <output-dir compare="Text">let28</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let29">
+          <output-dir compare="Text">let29</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let30">
+          <output-dir compare="Text">let30</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let31">
+          <output-dir compare="Text">let31</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="let32">
+          <output-dir compare="Text">let32</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-01">
+          <output-dir compare="Text">order-by-01</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-02">
+          <output-dir compare="Text">order-by-02</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-03">
+          <output-dir compare="Text">order-by-03</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-04">
+          <output-dir compare="Text">order-by-04</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-05">
+          <output-dir compare="Text">order-by-05</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-06">
+          <output-dir compare="Text">order-by-06</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-07">
+          <output-dir compare="Text">order-by-07</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-08">
+          <output-dir compare="Text">order-by-08</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-09">
+          <output-dir compare="Text">order-by-09</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-10">
+          <output-dir compare="Text">order-by-10</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-11">
+          <output-dir compare="Text">order-by-11</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="order-by-12">
+          <output-dir compare="Text">order-by-12</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-01">
+          <output-dir compare="Text">ret-01</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-02">
+          <output-dir compare="Text">ret-02</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-03">
+          <output-dir compare="Text">ret-03</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-04">
+          <output-dir compare="Text">ret-04</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-05">
+          <output-dir compare="Text">ret-05</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-06">
+          <output-dir compare="Text">ret-06</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-07">
+          <output-dir compare="Text">ret-07</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-08">
+          <output-dir compare="Text">ret-08</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-09">
+          <output-dir compare="Text">ret-09</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-10">
+          <output-dir compare="Text">ret-10</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-11">
+          <output-dir compare="Text">ret-11</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-12">
+          <output-dir compare="Text">ret-12</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-13">
+          <output-dir compare="Text">ret-13</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-14">
+          <output-dir compare="Text">ret-14</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-15">
+          <output-dir compare="Text">ret-15</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="writers">
+      <test-case FilePath="writers">
+        <compilation-unit name="print_01">
+          <output-dir compare="Text">print_01</output-dir>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-16">
+          <output-dir compare="Text">ret-16</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-17">
+          <output-dir compare="Text">ret-17</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-18">
+          <output-dir compare="Text">ret-18</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="flwor">
+        <compilation-unit name="ret-19">
+          <output-dir compare="Text">ret-19</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    -->
+  <test-group name="fulltext">
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-01">
+        <output-dir compare="Text">fulltext-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-02">
+        <output-dir compare="Text">fulltext-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-04">
+        <output-dir compare="Text">fulltext-04</output-dir>
+        <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-05">
+        <output-dir compare="Text">fulltext-05</output-dir>
+        <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-06">
+        <output-dir compare="Text">fulltext-06</output-dir>
+        <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-08">
+        <output-dir compare="Text">fulltext-08</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-09">
+        <output-dir compare="Text">fulltext-09</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-index-01">
+        <output-dir compare="Text">fulltext-index-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-index-02">
+        <output-dir compare="Text">fulltext-index-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-index-04">
+        <output-dir compare="Text">fulltext-index-04</output-dir>
+        <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-index-05">
+        <output-dir compare="Text">fulltext-index-05</output-dir>
+        <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-index-06">
+        <output-dir compare="Text">fulltext-index-06</output-dir>
+        <expected-error>ASX1010: Phrase search in Full-text is not yet supported. Only one keyword per expression is permitted</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-index-08">
+        <output-dir compare="Text">fulltext-index-08</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-index-09">
+        <output-dir compare="Text">fulltext-index-09</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="fulltext-index-large-data">
+        <output-dir compare="Text">fulltext-index-large-data</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fulltext">
+      <compilation-unit name="stopwords-full-text-filter-1">
+        <output-dir compare="Text">stopwords-full-text-filter-1</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="global-aggregate">
+    <test-case FilePath="global-aggregate">
+      <compilation-unit name="q01">
+        <output-dir compare="Text">q01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="global-aggregate">
+      <compilation-unit name="q02">
+        <output-dir compare="Text">q02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="global-aggregate">
+      <compilation-unit name="q03">
+        <output-dir compare="Text">q01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="global-aggregate">
+      <compilation-unit name="q04">
+        <output-dir compare="Text">q01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="global-aggregate">
+      <compilation-unit name="q05_error">
+        <output-dir compare="Text">q01</output-dir>
+        <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier u (in line 22, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="global-aggregate">
+      <compilation-unit name="q06_error">
+        <output-dir compare="Text">q01</output-dir>
+        <expected-error>ASX0037: Type mismatch: expected value of type array or multiset, but got the value of type string (in line 22, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="global-aggregate">
+      <compilation-unit name="q07_error">
+        <output-dir compare="Text">q01</output-dir>
+        <expected-error>count is a SQL-92 aggregate function. The SQL++ core aggregate function array_count could potentially express the intent.</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="global-aggregate">
+      <compilation-unit name="q08">
+        <output-dir compare="Text">q08</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="global-aggregate">
+      <compilation-unit name="q09">
+        <output-dir compare="Text">q09</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="global-aggregate">
+      <compilation-unit name="q10">
+        <output-dir compare="Text">q09</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="global-aggregate">
+      <compilation-unit name="q11">
+        <output-dir compare="Text">q01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="global-aggregate">
+      <compilation-unit name="q12_error">
+        <output-dir compare="Text">q01</output-dir>
+        <expected-error>The parameter * can only be used in count().</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="global-aggregate">
+      <compilation-unit name="query-ASTERIXDB-159">
+        <output-dir compare="Text">query-ASTERIXDB-159</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="global-aggregate">
+      <compilation-unit name="query-ASTERIXDB-1626">
+        <output-dir compare="Text">query-ASTERIXDB-1626</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="global-aggregate">
+      <compilation-unit name="query-ASTERIXDB-1626-2">
+        <output-dir compare="Text">query-ASTERIXDB-1626-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="global-aggregate">
+      <compilation-unit name="query-ASTERIXDB-2525">
+        <output-dir compare="Text">query-ASTERIXDB-2525</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="group-by">
+    <test-case FilePath="group-by">
+      <compilation-unit name="gby-array">
+        <output-dir compare="Text">gby-array</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="gby-case-01">
+        <output-dir compare="Text">gby-case-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="gby-cross-join">
+        <output-dir compare="Text">gby-cross-join</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="gby-nested-01">
+        <output-dir compare="Text">gby-nested-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="gby-record">
+        <output-dir compare="Text">gby-record</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="core-01">
+        <output-dir compare="Text">core-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="core-01-error">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>Cannot find dataset e in dataverse gby nor an alias with name e</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="core-02-error">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>Cannot find dataset f in dataverse gby nor an alias with name f</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="core-02">
+        <output-dir compare="Text">core-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="core-03">
+        <output-dir compare="Text">core-03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="core-04">
+        <output-dir compare="Text">core-04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="core-05">
+        <output-dir compare="Text">core-05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="core-06">
+        <output-dir compare="Text">core-06</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="core-07-error">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>GROUP BY x, y;<< Encountered "GROUP" at column 1]]></expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="sugar-01">
+        <output-dir compare="Text">core-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="sugar-01-negative">
+        <output-dir compare="Text">core-01</output-dir>
+        <expected-error>ASX0037: Type mismatch: expected value of type array or multiset, but got the value of type bigint (in line 26, at column 26)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="sugar-01-2">
+        <output-dir compare="Text">core-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="sugar-02">
+        <output-dir compare="Text">core-02</output-dir>
+        <expected-error>Cannot resolve ambiguous alias reference for identifier deptId</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="sugar-02-2">
+        <output-dir compare="Text">core-02</output-dir>
+        <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier deptId (in line 28, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="sugar-03">
+        <output-dir compare="Text">core-03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="sugar-03-2">
+        <output-dir compare="Text">core-03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="sugar-04">
+        <output-dir compare="Text">core-04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="sugar-04-2">
+        <output-dir compare="Text">core-04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="sugar-05">
+        <output-dir compare="Text">core-05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="sugar-05-2">
+        <output-dir compare="Text">core-05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="sugar-06-distinct">
+        <output-dir compare="Text">sugar-06-distinct</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="sugar-07-negative">
+        <output-dir compare="Text">core-01</output-dir>
+        <expected-error>Cannot resolve ambiguous alias reference for identifier IDENT</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="sugar-08-negative">
+        <output-dir compare="Text">core-01</output-dir>
+        <expected-error>ASX1103: Illegal use of identifier: x</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="sugar-09">
+        <output-dir compare="Text">sugar-09</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="two-step-agg-01">
+        <output-dir compare="Text">two-step-agg-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="null">
+        <output-dir compare="Text">null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="gby-expr">
+        <output-dir compare="Text">gby-expr</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="grouping-sets-1">
+        <output-dir compare="Text">grouping-sets-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="grouping-sets-2">
+        <output-dir compare="Text">grouping-sets-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="grouping-sets-3-negative">
+        <output-dir compare="Text">grouping-sets-2</output-dir>
+        <expected-error>ASX1120: Unexpected alias: v21</expected-error>
+        <expected-error>ASX1120: Unexpected alias: v22</expected-error>
+        <expected-error>ASX1120: Unexpected alias: v23</expected-error>
+        <expected-error>ASX1087: Invalid number of arguments for function grouping</expected-error>
+        <expected-error>ASX1119: Invalid argument to grouping() function</expected-error>
+        <expected-error>ASX1119: Invalid argument to grouping() function</expected-error>
+        <expected-error>ASX1119: Invalid argument to grouping() function</expected-error>
+        <expected-error>ASX1118: Too many grouping sets in group by clause: 512. Maximum allowed: 128.</expected-error>
+        <expected-error>ASX1129: Cannot compile SELECT variable.* with GROUP BY GROUPING SETS/ROLLUP/CUBE followed by ORDER BY/LIMIT</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="having">
+        <output-dir compare="Text">core-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="having-2">
+        <output-dir compare="Text">core-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="policy">
+        <output-dir compare="Text">policy</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="policy-02">
+        <output-dir compare="Text">policy-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="policy-03">
+        <output-dir compare="Text">policy-03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="policy-04">
+        <output-dir compare="Text">policy-04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by" check-warnings="true">
+      <compilation-unit name="policy-05">
+        <output-dir compare="Text">policy-05</output-dir>
+        <expected-warn>Unsupported type: agg-sum cannot process input type object (in line 29, at column 23)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="listify">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>The byte size of a single group</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="listify-2">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>The byte size of a single group</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="listify-3">
+        <output-dir compare="Text">listify-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="redundant-var-in-groupby">
+        <output-dir compare="Text">redundant-var-in-groupby</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="group-by-all-ASTERIXDB-2611">
+        <output-dir compare="Text">group-by-all-ASTERIXDB-2611</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
+      <compilation-unit name="hash-group-by-decor">
+        <output-dir compare="Text">hash-group-by-decor</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--test-case FilePath="group-by">
+      <compilation-unit name="query-ASTERIXDB-3016">
+        <output-dir compare="Text">query-ASTERIXDB-3016</output-dir>
+      </compilation-unit>
+    </test-case-->
+  </test-group>
+  <test-group name="index-join">
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-index-nested-loop-join">
+        <output-dir compare="Text">btree-index-nested-loop-join</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-primary-equi-join">
+        <output-dir compare="Text">btree-primary-equi-join</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-pidx-to-sidx-idxonly-equi-join_01">
+        <output-dir compare="Text">btree-pidx-to-sidx-idxonly-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-secondary-equi-join_01">
+        <output-dir compare="Text">btree-secondary-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-secondary-equi-join_02">
+        <output-dir compare="Text">btree-secondary-equi-join_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-secondary-equi-join_03">
+        <output-dir compare="Text">btree-secondary-equi-join_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-secondary-equi-join_04">
+        <output-dir compare="Text">btree-secondary-equi-join_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-sidx-idxonly-to-pidx-equi-join_01">
+        <output-dir compare="Text">btree-sidx-idxonly-to-pidx-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-sidx-idxonly-to-sidx-idxonly-equi-join_01">
+        <output-dir compare="Text">btree-sidx-idxonly-to-sidx-idxonly-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--test-case FilePath="index-join">
+      <compilation-unit name="btree-sidx-non-idxonly-to-pidx-equi-join_01">
+        <output-dir compare="Text">btree-sidx-non-idxonly-to-pidx-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-sidx-non-idxonly-to-sidx-idxonly-equi-join_01">
+        <output-dir compare="Text">btree-sidx-non-idxonly-to-sidx-idxonly-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case!-->
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-secondary-self-equi-join_01">
+        <output-dir compare="Text">btree-secondary-self-equi-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join" check-warnings="true">
+      <compilation-unit name="hints-indexnl-params">
+        <output-dir compare="Text">hints-indexnl-params</output-dir>
+        <expected-warn><![CDATA[ASX1132: Invalid specification for hint indexnl. ASX1001: Syntax error: In line 1 >>(8, idx_tenk2_1k_2k)<< Encountered <INTEGER_LITERAL> "8" at column 2.  (in line 35, at column 21)]]></expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="rtree-spatial-intersect-point_01">
+        <output-dir compare="Text">rtree-spatial-intersect-point_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="rtree-spatial-intersect-point_02">
+        <output-dir compare="Text">rtree-spatial-intersect-point_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="rtree-spatial-intersect-point_03">
+        <output-dir compare="Text">rtree-spatial-intersect-point_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="rtree-spatial-intersect-point_04">
+        <output-dir compare="Text">rtree-spatial-intersect-point_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="rtree-spatial-intersect-point_05">
+        <output-dir compare="Text">rtree-spatial-intersect-point_05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="rtree-spatial-self-intersect-point">
+        <output-dir compare="Text">rtree-spatial-self-intersect-point</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-join">
+      <compilation-unit name="btree-multiple-join">
+        <output-dir compare="Text">btree-multiple-join</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="index-selection">
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-index-composite-key">
+        <output-dir compare="Text">btree-index-composite-key</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-index-composite-key-02">
+        <output-dir compare="Text">btree-index-composite-key-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-index-composite-key-03">
+        <output-dir compare="Text">btree-index-composite-key-03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-index-composite-key-04">
+        <output-dir compare="Text">btree-index-composite-key-04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sec-primary-index">
+        <output-dir compare="Text">btree-sec-primary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sec-primary-index-01">
+        <output-dir compare="Text">btree-sec-primary-index-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sec-primary-index-02">
+        <output-dir compare="Text">btree-sec-primary-index-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sec-primary-index-03">
+        <output-dir compare="Text">btree-sec-primary-index-03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sec-primary-index-04">
+        <output-dir compare="Text">btree-sec-primary-index-04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-index-composite-key-mixed-intervals">
+        <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-index-rewrite-multiple">
+        <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sidx-composite-idxonly-01">
+        <output-dir compare="Text">btree-sidx-composite-idxonly-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sidx-composite-idxonly-02">
+        <output-dir compare="Text">btree-sidx-composite-idxonly-02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sidx-composite-idxonly-03">
+        <output-dir compare="Text">btree-sidx-composite-idxonly-03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sidx-composite-idxonly-04">
+        <output-dir compare="Text">btree-sidx-composite-idxonly-04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="btree-sidx-idxonly-01">
+        <output-dir compare="Text">btree-sidx-idxonly-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--test-case FilePath="index-selection">
+      <compilation-unit name="btree-sidx-non-idxonly-01">
+        <output-dir compare="Text">btree-sidx-non-idxonly-01</output-dir>
+      </compilation-unit>
+    </test-case!-->
+    <test-case FilePath="index-selection">
+      <compilation-unit name="cust-index-age-nullable">
+        <output-dir compare="Text">cust-index-age-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--test-case FilePath="index-selection">
+      <compilation-unit name="intersection-misc-01">
+        <output-dir compare="Text">intersection-misc-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="intersection-with-between">
+        <output-dir compare="Text">intersection-with-between</output-dir>
+      </compilation-unit>
+    </test-case!-->
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-ngram-contains">
+        <output-dir compare="Text">inverted-index-ngram-contains</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-ngram-edit-distance-panic">
+        <output-dir compare="Text">inverted-index-ngram-edit-distance-panic</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-ngram-edit-distance">
+        <output-dir compare="Text">inverted-index-ngram-edit-distance</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-ngram-edit-distance-large-data">
+        <output-dir compare="Text">inverted-index-ngram-edit-distance-large-data</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-ngram-edit-distance-word-tokens">
+        <output-dir compare="Text">inverted-index-ngram-edit-distance-word-tokens</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-ngram-jaccard">
+        <output-dir compare="Text">inverted-index-ngram-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-ngram-edit-distance-contains">
+        <output-dir compare="Text">inverted-index-ngram-edit-distance-contains</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-olist-edit-distance-panic">
+        <output-dir compare="Text">inverted-index-olist-edit-distance-panic</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-olist-edit-distance">
+        <output-dir compare="Text">inverted-index-olist-edit-distance</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-olist-jaccard">
+        <output-dir compare="Text">inverted-index-olist-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-ulist-jaccard">
+        <output-dir compare="Text">inverted-index-ulist-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-word-contains">
+        <output-dir compare="Text">inverted-index-word-contains</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-word-jaccard">
+        <output-dir compare="Text">inverted-index-word-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="orders-index-custkey-conjunctive-open">
+        <output-dir compare="Text">orders-index-custkey-conjunctive-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="orders-index-custkey-conjunctive">
+        <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="orders-index-custkey-open">
+        <output-dir compare="Text">orders-index-custkey-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="orders-index-custkey">
+        <output-dir compare="Text">orders-index-custkey</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="range-search-open">
+        <output-dir compare="Text">range-search-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="range-search">
+        <output-dir compare="Text">range-search</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="rtree-secondary-index-nullable">
+        <output-dir compare="Text">rtree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="rtree-secondary-index-open">
+        <output-dir compare="Text">rtree-secondary-index-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="rtree-secondary-index">
+        <output-dir compare="Text">rtree-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="rtree-secondary-index-circular-query">
+        <output-dir compare="Text">rtree-secondary-index-circular-query</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="rtree-sidx-idxonly-01">
+        <output-dir compare="Text">rtree-sidx-idxonly-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--test-case FilePath="index-selection">
+      <compilation-unit name="rtree-sidx-non-idxonly-01">
+        <output-dir compare="Text">rtree-sidx-non-idxonly-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="rtree-sidx-non-idxonly-02">
+        <output-dir compare="Text">rtree-sidx-non-idxonly-02</output-dir>
+      </compilation-unit>
+    </test-case!-->
+    <test-case FilePath="index-selection">
+      <compilation-unit name="disjunctive-predicate-1">
+        <output-dir compare="Text">disjunctive-predicate-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="dataset-with-meta">
+        <output-dir compare="Text">dataset-with-meta</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--test-case FilePath="index-selection" check-warnings="true">
+      <compilation-unit name="hints-skip-index">
+        <output-dir compare="Text">hints-skip-index</output-dir>
+        <expected-warn><![CDATA[ASX1132: Invalid specification for hint skip-index. ASX1001: Syntax error: In line 1 >>(13, idx_1k)<< Encountered <INTEGER_LITERAL> "13" at column 2.  (in line 32, at column 19)]]></expected-warn>
+      </compilation-unit>
+    </test-case!-->
+    <!--test-case FilePath="index-selection" check-warnings="true">
+      <compilation-unit name="hints-use-index">
+        <output-dir compare="Text">hints-use-index</output-dir>
+        <expected-warn><![CDATA[ASX1132: Invalid specification for hint use-index. ASX1001: Syntax error: In line 1 >>(18, idx_1k_2k)<< Encountered <INTEGER_LITERAL> "18" at column 2.  (in line 33, at column 15)]]></expected-warn>
+        <expected-warn><![CDATA[ASX1132: Invalid specification for hint use-index. ASX1001: Syntax error: In line 1 >>()<< Encountered ")" at column 2.  (in line 33, at column 15)]]></expected-warn>
+      </compilation-unit>
+    </test-case!-->
+    <test-case FilePath="index-selection">
+      <compilation-unit name="inverted-index-ngram-edit-distance-with-two-ngram-index">
+        <output-dir compare="Text">inverted-index-ngram-edit-distance-with-two-ngram-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--test-case FilePath="index-selection">
+      <compilation-unit name="intersection">
+        <output-dir compare="Text">intersection</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="intersection-with-filter">
+        <output-dir compare="Text">intersection-with-filter</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="intersection_with_nodegroup">
+        <output-dir compare="Text">intersection</output-dir>
+      </compilation-unit>
+    </test-case!-->
+    <test-case FilePath="index-selection">
+      <compilation-unit name="verify">
+        <output-dir compare="Text">verify</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="inverted-index-join">
+    <test-case FilePath="inverted-index-join">
+      <compilation-unit name="ngram-edit-distance">
+        <output-dir compare="Text">ngram-edit-distance</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join">
+      <compilation-unit name="ngram-edit-distance-inline">
+        <output-dir compare="Text">ngram-edit-distance-inline</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join">
+      <compilation-unit name="ngram-jaccard">
+        <output-dir compare="Text">ngram-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join">
+      <compilation-unit name="ngram-jaccard-inline">
+        <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join">
+      <compilation-unit name="olist-edit-distance">
+        <output-dir compare="Text">olist-edit-distance</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join">
+      <compilation-unit name="olist-edit-distance-inline">
+        <output-dir compare="Text">olist-edit-distance-inline</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join">
+      <compilation-unit name="olist-jaccard">
+        <output-dir compare="Text">olist-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join">
+      <compilation-unit name="olist-jaccard-inline">
+        <output-dir compare="Text">olist-jaccard-inline</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join">
+      <compilation-unit name="ulist-jaccard">
+        <output-dir compare="Text">ulist-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join">
+      <compilation-unit name="ulist-jaccard-inline">
+        <output-dir compare="Text">ulist-jaccard-inline</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join">
+      <compilation-unit name="word-jaccard">
+        <output-dir compare="Text">word-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join">
+      <compilation-unit name="word-jaccard-inline">
+        <output-dir compare="Text">word-jaccard-inline</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="cast-default-null">
+        <placeholder name="with" value="" />
+        <output-dir compare="Text">cast-default-null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="cast-default-null">
+        <placeholder name="with" value="WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" />
+        <output-dir compare="Text">cast-default-null</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="inverted-index-join-noeqjoin">
+    <test-case FilePath="inverted-index-join-noeqjoin">
+      <compilation-unit name="ngram-edit-distance">
+        <output-dir compare="Text">ngram-edit-distance</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join-noeqjoin">
+      <compilation-unit name="ngram-edit-distance-inline">
+        <output-dir compare="Text">ngram-edit-distance-inline</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join-noeqjoin">
+      <compilation-unit name="ngram-jaccard">
+        <output-dir compare="Text">ngram-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join-noeqjoin">
+      <compilation-unit name="ngram-jaccard-inline">
+        <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join-noeqjoin">
+      <compilation-unit name="olist-edit-distance">
+        <output-dir compare="Text">olist-edit-distance</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join-noeqjoin">
+      <compilation-unit name="olist-edit-distance-inline">
+        <output-dir compare="Text">olist-edit-distance-inline</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join-noeqjoin">
+      <compilation-unit name="olist-jaccard">
+        <output-dir compare="Text">olist-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join-noeqjoin">
+      <compilation-unit name="olist-jaccard-inline">
+        <output-dir compare="Text">olist-jaccard-inline</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join-noeqjoin">
+      <compilation-unit name="ulist-jaccard">
+        <output-dir compare="Text">ulist-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join-noeqjoin">
+      <compilation-unit name="ulist-jaccard-inline">
+        <output-dir compare="Text">ulist-jaccard-inline</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join-noeqjoin">
+      <compilation-unit name="word-jaccard">
+        <output-dir compare="Text">word-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="inverted-index-join-noeqjoin">
+      <compilation-unit name="word-jaccard-inline">
+        <output-dir compare="Text">word-jaccard-inline</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="join">
+    <test-case FilePath="join">
+      <compilation-unit name="cross-join-01">
+        <output-dir compare="Text">cross-join-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="join">
+      <compilation-unit name="cross-join-02-negative">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1077: Cannot find dataset x in dataverse Default nor an alias with name x (in line 26, at column 39)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="join">
+      <compilation-unit name="hash_join_array">
+        <output-dir compare="Text">hash_join_array</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="join">
+      <compilation-unit name="hash_join_missing">
+        <output-dir compare="Text">hash_join_missing</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="join">
+      <compilation-unit name="hash_join_record">
+        <output-dir compare="Text">hash_join_record</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="join">
+      <compilation-unit name="join-with-empty-dataset">
+        <output-dir compare="Text">join-with-empty-dataset</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="list">
+    <test-case FilePath="list">
+      <compilation-unit name="any-collection-member_01">
+        <output-dir compare="Text">any-collection-member_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="array_length">
+        <output-dir compare="Text">array_length</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="enforcing_item_type">
+        <output-dir compare="Text">enforcing_item_type</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="exists">
+        <output-dir compare="Text">exists</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="get-item_01">
+        <output-dir compare="Text">get-item_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="get-item_02">
+        <output-dir compare="Text">get-item_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="get-item_03">
+        <output-dir compare="Text">get-item_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="len_01">
+        <output-dir compare="Text">len_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="len_null_01">
+        <output-dir compare="Text">len_null_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="list-range">
+        <output-dir compare="Text">list-range</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="list-slice_01">
+        <output-dir compare="Text">list-slice_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="list-slice_02">
+        <output-dir compare="Text">list-slice_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="list-slice_03">
+        <output-dir compare="Text">list-slice_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="list-star_01">
+        <output-dir compare="Text">list-star_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="listify_01">
+        <output-dir compare="Text">listify_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="listify_02">
+        <output-dir compare="Text">listify_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="listify_03">
+        <output-dir compare="Text">listify_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="ordered-list-constructor_01">
+        <output-dir compare="Text">ordered-list-constructor_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="ordered-list-constructor_02">
+        <output-dir compare="Text">ordered-list-constructor_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="ordered-list-constructor_03">
+        <output-dir compare="Text">ordered-list-constructor_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="ordered-list-constructor_04">
+        <output-dir compare="Text">ordered-list-constructor_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="ordered-list-constructor_05">
+        <output-dir compare="Text">ordered-list-constructor_05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="ordered-list-constructor_06_negative">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error><![CDATA[ASX1001: Syntax error: In line 26 >>    from [r, r+1] x select value count(x),<< Encountered "," at column 42]]></expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="scan-collection_01">
+        <output-dir compare="Text">scan-collection_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--         <test-case FilePath="list">
+                    <compilation-unit name="union_01">
+                        <output-dir compare="Text">union_01</output-dir>
+                    </compilation-unit>
+                </test-case>
+                <test-case FilePath="list">
+                    <compilation-unit name="union_02">
+                        <output-dir compare="Text">union_02</output-dir>
+                    </compilation-unit>
+                </test-case>
+         -->
+    <test-case FilePath="list">
+      <compilation-unit name="unordered-list-constructor_01">
+        <output-dir compare="Text">unordered-list-constructor_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="unordered-list-constructor_02">
+        <output-dir compare="Text">unordered-list-constructor_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="unordered-list-constructor_03">
+        <output-dir compare="Text">unordered-list-constructor_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="query-issue428">
+        <output-dir compare="Text">query-issue428</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="query-ASTERIXDB-159-2">
+        <output-dir compare="Text">query-ASTERIXDB-159-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="query-ASTERIXDB-159-3">
+        <output-dir compare="Text">query-ASTERIXDB-159-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="query-ASTERIXDB-1131-2">
+        <output-dir compare="Text">query-ASTERIXDB-1131-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="query-ASTERIXDB-1131">
+        <output-dir compare="Text">query-ASTERIXDB-1131</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="query-ASTERIXDB-1212-2-open">
+        <output-dir compare="Text">query-ASTERIXDB-1212-2-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="query-ASTERIXDB-1212-2">
+        <output-dir compare="Text">query-ASTERIXDB-1212-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="query-ASTERIXDB-1212-open">
+        <output-dir compare="Text">query-ASTERIXDB-1212-open</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="query-ASTERIXDB-1212">
+        <output-dir compare="Text">query-ASTERIXDB-1212</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="query-ASTERIXDB-673">
+        <output-dir compare="Text">query-ASTERIXDB-673</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="var-in-list">
+        <output-dir compare="Text">var-in-list</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="misc">
+    <test-case FilePath="misc">
+      <compilation-unit name="big_in_list/000">
+        <output-dir compare="Text">big_in_list/000</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="big_in_list/001">
+        <output-dir compare="Text">big_in_list/001</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="multiple_and/000">
+        <output-dir compare="Text">multiple_and/000</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="record-serialization-ASTERIXDB-2567">
+        <output-dir compare="Text">record-serialization-ASTERIXDB-2567</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="record-serialization-ASTERIXDB-2613">
+        <output-dir compare="Text">record-serialization-ASTERIXDB-2613</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="field_access-ASTERIXDB-2289">
+        <output-dir compare="Text">field_access-ASTERIXDB-2289</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="comp-ASTERIXDB-2415">
+        <output-dir compare="Text">query-ASTERIXDB-1671</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="comp-ASTERIXDB-2412">
+        <output-dir compare="Text">comp-ASTERIXDB-2412</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="field_access_union-ASTERIXDB-2288">
+        <output-dir compare="Text">field_access_union-ASTERIXDB-2288</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="constant_folding">
+        <output-dir compare="Text">constant_folding</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="join-ASTERIXDB-2686">
+        <output-dir compare="Text">join-ASTERIXDB-2686</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="poll-dynamic">
+        <output-dir compare="Text">poll-dynamic</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="validate-expected">
+        <output-dir compare="Text">validate-expected</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="dataset-resources">
+        <output-dir compare="Text">dataset-resources</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="ping">
+        <output-dir compare="Text">ping</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="case_01">
+        <output-dir compare="Text">case_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="case_02">
+        <output-dir compare="Text">case_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="case_03">
+        <output-dir compare="Text">case_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="case_04">
+        <output-dir compare="Text">case_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="case_05">
+        <output-dir compare="Text">case_05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="case_06">
+        <output-dir compare="Text">case_06</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="case_07">
+        <output-dir compare="Text">case_05</output-dir>
+      </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="case_09">
+        <output-dir compare="Text">case_09</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>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="ensure_result_numeric_type">
+        <output-dir compare="Text">ensure_result_numeric_type</output-dir>
+        <expected-error>expected &lt; 3.0</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <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 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>
+    <test-case FilePath="misc">
+      <compilation-unit name="float_01">
+        <output-dir compare="Text">float_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="flushtest">
+        <output-dir compare="Text">flushtest</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="groupby-orderby-count">
+        <output-dir compare="Text">groupby-orderby-count</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="identifier_01">
+        <output-dir compare="Text">identifier_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="ifthenelse_01">
+        <output-dir compare="Text">ifthenelse_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="is-null_01">
+        <output-dir compare="Text">is-null_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="nested-loop-join_01">
+        <output-dir compare="Text">nested-loop-join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query_issue267">
+        <output-dir compare="Text">query_issue267</output-dir>
+      </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>
+    </test-case>
+    <!--
+         <test-case FilePath="misc">
+           <compilation-unit name="range_01">
+             <output-dir compare="Text">range_01</output-dir>
+           </compilation-unit>
+         </test-case>
+        -->
+    <!--
+          <test-case FilePath="misc">
+            <compilation-unit name="tid_01">
+              <output-dir compare="Text">tid_01</output-dir>
+            </compilation-unit>
+          </test-case>
+         -->
+    <test-case FilePath="misc">
+      <compilation-unit name="year_01">
+        <output-dir compare="Text">year_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="string_eq_01">
+        <output-dir compare="Text">string_eq_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="prefix-search">
+        <output-dir compare="Text">prefix-search</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-1490">
+        <output-dir compare="Text">query-ASTERIXDB-1490</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-971">
+        <output-dir compare="Text">query-ASTERIXDB-971-sqlpp</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-1531">
+        <output-dir compare="Text">query-ASTERIXDB-1531</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-1577">
+        <output-dir compare="Text">query-ASTERIXDB-1577</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-1671">
+        <output-dir compare="Text">query-ASTERIXDB-1671</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-1671-2">
+        <output-dir compare="Text">query-ASTERIXDB-1671</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-2354">
+        <output-dir compare="Text">query-ASTERIXDB-2354</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-2355">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error><![CDATA[ASX1001: Syntax error: In line 22 >> %%%<< Encountered "%" at column 2.]]></expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-2380">
+        <output-dir compare="Text">query-ASTERIXDB-2380</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-2550">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX0037: Type mismatch: expected value of type array or multiset, but got the value of type object (in line 28, at column 2)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-2886">
+        <output-dir compare="Text">query-ASTERIXDB-2886</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="unsupported_parameter">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>Query parameter compiler.joinmem is not supported</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="unsupported_parameter_value">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>The given string: 4LS is not a byte unit string (e.g., 320KB or 1024)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="uuid">
+        <output-dir compare="Text">uuid</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="p_sort_join">
+        <output-dir compare="Text">p_sort_join</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="p_sort_seq_merge">
+        <output-dir compare="Text">p_sort_seq_merge</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="p_sort_num_samples">
+        <output-dir compare="Text">p_sort_num_samples</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="p_sort_static_range_map">
+        <output-dir compare="Text">p_sort_static_range_map</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="active_requests">
+        <output-dir compare="Text">active_requests</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="jobs">
+        <output-dir compare="Text">jobs</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="completed_requests">
+        <output-dir compare="Text">completed_requests</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="dump_index">
+        <output-dir compare="Text">dump_index</output-dir>
+        <expected-error>Cannot find index with name noindex</expected-error>
+        <expected-error>Cannot find dataset with name nodataset in dataverse test</expected-error>
+        <expected-error>Cannot find dataset with name ds in dataverse nodataverse</expected-error>
+        <expected-error>Unsupported type: dump-index cannot process input type null</expected-error>
+        <expected-error>Unsupported type: dump-index cannot process input type null</expected-error>
+        <expected-error>Unsupported type: dump-index cannot process input type null</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="metadata_only_01">
+        <output-dir compare="Text">metadata_only_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="cast-ASTERIXDB-2458">
+        <output-dir compare="Text">cast-ASTERIXDB-2458</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="insert_nulls_with_secondary_idx">
+        <output-dir compare="Text">insert_nulls_with_secondary_idx</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-2700">
+        <output-dir compare="Text">query-ASTERIXDB-2700</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-1203">
+        <output-dir compare="Text">query-ASTERIXDB-1203</output-dir>
+      </compilation-unit>
+    </test-case>
+    -->
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-819-2">
+        <output-dir compare="Text">query-ASTERIXDB-819-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-819">
+        <output-dir compare="Text">query-ASTERIXDB-819</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-865">
+        <output-dir compare="Text">query-ASTERIXDB-865</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="multipart-dataverse">
+    <test-case FilePath="multipart-dataverse">
+      <compilation-unit name="index_1">
+        <output-dir compare="Text">index_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="multipart-dataverse">
+      <compilation-unit name="resolution_1">
+        <output-dir compare="Text">resolution_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="multipart-dataverse">
+      <compilation-unit name="special_chars_1">
+        <output-dir compare="Text">special_chars_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="multipart-dataverse">
+      <compilation-unit name="special_chars_2">
+        <output-dir compare="Text">special_chars_2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="multipart-dataverse">
+      <compilation-unit name="udf_1">
+        <output-dir compare="Text">udf_1</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="index">
+    <test-group name="index/validations">
+      <test-case FilePath="index/validations">
+        <compilation-unit name="keys-same-as-pk-but-different-order">
+          <output-dir compare="Text">keys-same-as-pk-but-different-order</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="index/validations">
+        <compilation-unit name="keys-same-as-pk-in-same-order">
+          <output-dir compare="Text">keys-same-as-pk-in-same-order</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="index/validations">
+        <compilation-unit name="repetitive-keys">
+          <output-dir compare="Text">repetitive-keys</output-dir>
+          <expected-error>Cannot create index with the same field '[value]' specified more than once.</expected-error>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+  </test-group>
+  <test-group name="open-index-enforced">
+    <test-group name="open-index-enforced/error-checking">
+      <test-case FilePath="open-index-enforced/error-checking">
+        <compilation-unit name="enforced-field-name-collision">
+          <output-dir compare="Text">enforced-field-name-collision</output-dir>
+          <expected-error>Cannot create enforced index on '[value]' field. The field is closed type.</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/error-checking">
+        <compilation-unit name="enforced-field-type-collision">
+          <output-dir compare="Text">enforced-field-type-collision</output-dir>
+          <expected-error>Cannot create enforced index on '[value]' field. The field is closed type.</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/error-checking">
+        <compilation-unit name="missing-enforce-statement">
+          <output-dir compare="Text">missing-enforce-statement</output-dir>
+          <expected-error>ASX1042: Cannot create non-enforced typed index of this kind: RTREE</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/error-checking">
+        <compilation-unit name="missing-optionality">
+          <output-dir compare="Text">missing-optionality</output-dir>
+          <expected-error>Cannot create enforced index on '[value]' field with non-optional type</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/error-checking">
+        <compilation-unit name="index-on-closed-type">
+          <output-dir compare="Text">index-on-closed-type</output-dir>
+          <expected-error>ASX1014: Field 'value' is not found (in line 33, at column 34)</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/error-checking">
+        <compilation-unit name="index-type-collision">
+          <output-dir compare="Text">index-type-collision</output-dir>
+          <expected-error>Cannot create index testIdx2 , enforced index testIdx1 on field(s) 'value' is already defined with type(s) 'integer'</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/error-checking">
+        <compilation-unit name="index-type-promotion-collision">
+          <output-dir compare="Text">index-type-promotion-collision</output-dir>
+          <expected-error>Cannot create index testIdx2 , enforced index testIdx1 on field(s) 'value' is already defined with type(s) 'bigint'</expected-error>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/error-checking">
+        <compilation-unit name="object-type-collision">
+          <output-dir compare="Text">object-type-collision</output-dir>
+          <expected-error>ASX1051: Cannot create enforced index on '[value]' field. The field is closed type.</expected-error>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="open-index-enforced/index-join">
+      <test-case FilePath="open-index-enforced/index-join">
+        <compilation-unit name="btree-secondary-equi-join">
+          <output-dir compare="Text">btree-secondary-equi-join</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-join">
+        <compilation-unit name="ngram-edit-distance">
+          <output-dir compare="Text">ngram-edit-distance</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-join">
+        <compilation-unit name="ngram-edit-distance-inline">
+          <output-dir compare="Text">ngram-edit-distance-inline</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-join">
+        <compilation-unit name="ngram-jaccard">
+          <output-dir compare="Text">ngram-jaccard</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-join">
+        <compilation-unit name="ngram-jaccard-inline">
+          <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-join">
+        <compilation-unit name="rtree-spatial-intersect-point">
+          <output-dir compare="Text">rtree-spatial-intersect-point</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-join">
+        <compilation-unit name="word-jaccard">
+          <output-dir compare="Text">word-jaccard</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-join">
+        <compilation-unit name="word-jaccard-inline">
+          <output-dir compare="Text">word-jaccard-inline</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="open-index-enforced/index-leftouterjoin">
+      <test-case FilePath="open-index-enforced/index-leftouterjoin">
+        <compilation-unit name="probe-pidx-with-join-btree-sidx1">
+          <output-dir compare="Text">probe-pidx-with-join-btree-sidx1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-leftouterjoin">
+        <compilation-unit name="probe-pidx-with-join-btree-sidx2">
+          <output-dir compare="Text">probe-pidx-with-join-btree-sidx2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-leftouterjoin">
+        <compilation-unit name="probe-pidx-with-join-invidx-sidx2">
+          <output-dir compare="Text">probe-pidx-with-join-invidx-sidx2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-leftouterjoin">
+        <compilation-unit name="probe-pidx-with-join-rtree-sidx1">
+          <output-dir compare="Text">probe-pidx-with-join-rtree-sidx1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-leftouterjoin">
+        <compilation-unit name="probe-pidx-with-join-rtree-sidx2">
+          <output-dir compare="Text">probe-pidx-with-join-rtree-sidx2</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="open-index-enforced/index-selection">
+      <test-case FilePath="open-index-enforced/index-selection">
+        <compilation-unit name="btree-index-composite-key">
+          <output-dir compare="Text">btree-index-composite-key</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-selection">
+        <compilation-unit name="btree-index-composite-key-mixed-intervals">
+          <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-selection">
+        <compilation-unit name="btree-index-rewrite-multiple">
+          <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-selection">
+        <compilation-unit name="inverted-index-ngram-contains">
+          <output-dir compare="Text">inverted-index-ngram-contains</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-selection">
+        <compilation-unit name="inverted-index-ngram-edit-distance">
+          <output-dir compare="Text">inverted-index-ngram-edit-distance</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-selection">
+        <compilation-unit name="inverted-index-ngram-edit-distance-contains">
+          <output-dir compare="Text">inverted-index-ngram-edit-distance-contains</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-selection">
+        <compilation-unit name="inverted-index-ngram-edit-distance-panic">
+          <output-dir compare="Text">inverted-index-ngram-edit-distance-panic</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-selection">
+        <compilation-unit name="inverted-index-ngram-edit-distance-word-tokens">
+          <output-dir compare="Text">inverted-index-ngram-edit-distance-word-tokens</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-selection">
+        <compilation-unit name="inverted-index-ngram-jaccard">
+          <output-dir compare="Text">inverted-index-ngram-jaccard</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-selection">
+        <compilation-unit name="inverted-index-word-contains">
+          <output-dir compare="Text">inverted-index-word-contains</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-selection">
+        <compilation-unit name="inverted-index-word-jaccard">
+          <output-dir compare="Text">inverted-index-word-jaccard</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-selection">
+        <compilation-unit name="orders-index-custkey">
+          <output-dir compare="Text">orders-index-custkey</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-selection">
+        <compilation-unit name="orders-index-custkey-conjunctive">
+          <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-selection">
+        <compilation-unit name="range-search">
+          <output-dir compare="Text">range-search</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-selection">
+        <compilation-unit name="rtree-secondary-index">
+          <output-dir compare="Text">rtree-secondary-index</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-selection">
+        <compilation-unit name="multi-index-composite-key">
+          <output-dir compare="Text">multi-index-composite-key</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/index-selection">
+        <compilation-unit name="multi-index">
+          <output-dir compare="Text">multi-index</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="open-index-enforced/type-checking">
+      <test-case FilePath="open-index-enforced/type-checking">
+        <compilation-unit name="enforced-type-delete">
+          <output-dir compare="Text">enforced-type-delete</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/type-checking">
+        <compilation-unit name="enforced-type-upsert">
+          <output-dir compare="Text">enforced-type-upsert</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="open-index-non-enforced/index-selection">
+      <test-case FilePath="open-index-non-enforced/index-selection">
+        <compilation-unit name="btree-index-01">
+          <output-dir compare="Text">btree-index-01</output-dir>
+        </compilation-unit>
+      </test-case>
+      <!--test-case FilePath="open-index-non-enforced/index-selection">
+        <compilation-unit name="btree-index-02">
+          <output-dir compare="Text">btree-index-02</output-dir>
+        </compilation-unit>
+      </test-case!-->
+      <test-case FilePath="open-index-non-enforced/index-selection">
+        <compilation-unit name="btree-index-03">
+          <output-dir compare="Text">btree-index-03</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-non-enforced/index-selection">
+        <compilation-unit name="btree-index-04">
+          <output-dir compare="Text">btree-index-04</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-non-enforced/index-selection">
+        <compilation-unit name="btree-index-composite-key-03">
+          <output-dir compare="Text">btree-index-composite-key-03</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-non-enforced/index-selection">
+        <compilation-unit name="btree-index-composite-key-04">
+          <output-dir compare="Text">btree-index-composite-key-04</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-non-enforced/correlated-index-selection">
+        <compilation-unit name="btree-index-01">
+          <output-dir compare="Text">btree-index-01</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="open-index-non-enforced/index-join">
+      <test-case FilePath="open-index-non-enforced/index-join">
+        <compilation-unit name="btree-equi-join-01">
+          <output-dir compare="Text">btree-equi-join-01</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+  </test-group>
+  <test-group name="nested-open-index">
+    <test-group name="nested-open-index/index-join">
+      <test-case FilePath="nested-open-index/index-join">
+        <compilation-unit name="btree-secondary-equi-join">
+          <output-dir compare="Text">btree-secondary-equi-join</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-join">
+        <compilation-unit name="ngram-edit-distance">
+          <output-dir compare="Text">ngram-edit-distance</output-dir>
+        </compilation-unit>
+      </test-case>
+      <!--  <test-case FilePath="nested-open-index/index-join">
+                <compilation-unit name="ngram-edit-distance-inline">
+                    <output-dir compare="Text">ngram-edit-distance-inline</output-dir>
+                </compilation-unit>
+            </test-case> -->
+      <test-case FilePath="nested-open-index/index-join">
+        <compilation-unit name="ngram-jaccard">
+          <output-dir compare="Text">ngram-jaccard</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-join">
+        <compilation-unit name="ngram-jaccard-inline">
+          <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-join">
+        <compilation-unit name="rtree-spatial-intersect-point">
+          <output-dir compare="Text">rtree-spatial-intersect-point</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-join">
+        <compilation-unit name="word-jaccard">
+          <output-dir compare="Text">word-jaccard</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-join">
+        <compilation-unit name="word-jaccard-inline">
+          <output-dir compare="Text">word-jaccard-inline</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-join">
+        <compilation-unit name="btree-secondary-non-enforced-equi-join">
+          <output-dir compare="Text">btree-secondary-non-enforced-equi-join</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-join">
+        <compilation-unit name="btree-secondary-non-enforced-equi-join-2">
+          <output-dir compare="Text">btree-secondary-non-enforced-equi-join-2</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="nested-open-index/index-leftouterjoin">
+      <test-case FilePath="nested-open-index/index-leftouterjoin">
+        <compilation-unit name="probe-pidx-with-join-btree-sidx1">
+          <output-dir compare="Text">probe-pidx-with-join-btree-sidx1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-leftouterjoin">
+        <compilation-unit name="probe-pidx-with-join-btree-sidx2">
+          <output-dir compare="Text">probe-pidx-with-join-btree-sidx2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-leftouterjoin">
+        <compilation-unit name="probe-pidx-with-join-invidx-sidx2">
+          <output-dir compare="Text">probe-pidx-with-join-invidx-sidx2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-leftouterjoin">
+        <compilation-unit name="probe-pidx-with-join-rtree-sidx1">
+          <output-dir compare="Text">probe-pidx-with-join-rtree-sidx1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-leftouterjoin">
+        <compilation-unit name="probe-pidx-with-join-rtree-sidx2">
+          <output-dir compare="Text">probe-pidx-with-join-rtree-sidx2</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="nested-open-index/index-selection">
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="btree-index-composite-key">
+          <output-dir compare="Text">btree-index-composite-key</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="btree-index-composite-key-mixed-intervals">
+          <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="btree-index-rewrite-multiple">
+          <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="inverted-index-ngram-contains">
+          <output-dir compare="Text">inverted-index-ngram-contains</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="inverted-index-ngram-edit-distance">
+          <output-dir compare="Text">inverted-index-ngram-edit-distance</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="inverted-index-ngram-edit-distance-contains">
+          <output-dir compare="Text">inverted-index-ngram-edit-distance-contains</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="inverted-index-ngram-edit-distance-panic">
+          <output-dir compare="Text">inverted-index-ngram-edit-distance-panic</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="inverted-index-ngram-edit-distance-word-tokens">
+          <output-dir compare="Text">inverted-index-ngram-edit-distance-word-tokens</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="inverted-index-ngram-jaccard">
+          <output-dir compare="Text">inverted-index-ngram-jaccard</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="inverted-index-word-contains">
+          <output-dir compare="Text">inverted-index-word-contains</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="inverted-index-word-jaccard">
+          <output-dir compare="Text">inverted-index-word-jaccard</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="orders-index-custkey">
+          <output-dir compare="Text">orders-index-custkey</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="orders-index-custkey-conjunctive">
+          <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="range-search">
+          <output-dir compare="Text">range-search</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="rtree-secondary-index">
+          <output-dir compare="Text">rtree-secondary-index</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="non-enforced-01">
+          <output-dir compare="Text">non-enforced-01</output-dir>
+        </compilation-unit>
+      </test-case>
+      <!--test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="non-enforced-02">
+          <output-dir compare="Text">non-enforced-02</output-dir>
+        </compilation-unit>
+      </test-case!-->
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="non-enforced-03">
+          <output-dir compare="Text">non-enforced-03</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/index-selection">
+        <compilation-unit name="non-enforced-04">
+          <output-dir compare="Text">non-enforced-04</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="nested-open-index/highly-open-highly-nested">
+      <test-case FilePath="nested-open-index/highly-open-highly-nested">
+        <compilation-unit name="bottom-closed-top-closed">
+          <output-dir compare="Text">bottom-closed-top-closed</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/highly-open-highly-nested">
+        <compilation-unit name="bottom-closed-top-open">
+          <output-dir compare="Text">bottom-closed-top-open</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/highly-open-highly-nested">
+        <compilation-unit name="bottom-open-top-closed">
+          <output-dir compare="Text">bottom-open-top-closed</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-open-index/highly-open-highly-nested">
+        <compilation-unit name="bottom-open-top-open">
+          <output-dir compare="Text">bottom-open-top-open</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+  </test-group>
+  <test-group name="nested-index">
+    <test-group name="nested-index/index-join">
+      <test-case FilePath="nested-index/index-join">
+        <compilation-unit name="btree-primary-equi-join">
+          <output-dir compare="Text">btree-primary-equi-join</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-join">
+        <compilation-unit name="btree-secondary-equi-join">
+          <output-dir compare="Text">btree-secondary-equi-join</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-join">
+        <compilation-unit name="ngram-edit-distance">
+          <output-dir compare="Text">ngram-edit-distance</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-join">
+        <compilation-unit name="ngram-edit-distance-inline">
+          <output-dir compare="Text">ngram-edit-distance-inline</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-join">
+        <compilation-unit name="ngram-jaccard">
+          <output-dir compare="Text">ngram-jaccard</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-join">
+        <compilation-unit name="ngram-jaccard-inline">
+          <output-dir compare="Text">ngram-jaccard-inline</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-join">
+        <compilation-unit name="rtree-spatial-intersect-point">
+          <output-dir compare="Text">rtree-spatial-intersect-point</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-join">
+        <compilation-unit name="word-jaccard">
+          <output-dir compare="Text">word-jaccard</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-join">
+        <compilation-unit name="word-jaccard-inline">
+          <output-dir compare="Text">word-jaccard-inline</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="nested-index/index-leftouterjoin">
+      <test-case FilePath="nested-index/index-leftouterjoin">
+        <compilation-unit name="probe-pidx-with-join-btree-sidx1">
+          <output-dir compare="Text">probe-pidx-with-join-btree-sidx1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-leftouterjoin">
+        <compilation-unit name="probe-pidx-with-join-btree-sidx2">
+          <output-dir compare="Text">probe-pidx-with-join-btree-sidx2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-leftouterjoin">
+        <compilation-unit name="probe-pidx-with-join-invidx-sidx1">
+          <output-dir compare="Text">probe-pidx-with-join-invidx-sidx1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-leftouterjoin">
+        <compilation-unit name="probe-pidx-with-join-invidx-sidx2">
+          <output-dir compare="Text">probe-pidx-with-join-invidx-sidx2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-leftouterjoin">
+        <compilation-unit name="probe-pidx-with-join-rtree-sidx1">
+          <output-dir compare="Text">probe-pidx-with-join-rtree-sidx1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-leftouterjoin">
+        <compilation-unit name="probe-pidx-with-join-rtree-sidx2">
+          <output-dir compare="Text">probe-pidx-with-join-rtree-sidx2</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="nested-index/index-selection">
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="btree-index-composite-key">
+          <output-dir compare="Text">btree-index-composite-key</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="btree-sec-primary-index">
+          <output-dir compare="Text">btree-sec-primary-index</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="btree-index-composite-key-mixed-intervals">
+          <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="btree-index-rewrite-multiple">
+          <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="cust-index-age-nullable">
+          <output-dir compare="Text">cust-index-age-nullable</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="inverted-index-ngram-contains">
+          <output-dir compare="Text">inverted-index-ngram-contains</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="inverted-index-ngram-edit-distance">
+          <output-dir compare="Text">inverted-index-ngram-edit-distance</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="inverted-index-ngram-edit-distance-contains">
+          <output-dir compare="Text">inverted-index-ngram-edit-distance-contains</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="inverted-index-ngram-edit-distance-panic">
+          <output-dir compare="Text">inverted-index-ngram-edit-distance-panic</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="inverted-index-ngram-edit-distance-word-tokens">
+          <output-dir compare="Text">inverted-index-ngram-edit-distance-word-tokens</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="inverted-index-ngram-jaccard">
+          <output-dir compare="Text">inverted-index-ngram-jaccard</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="inverted-index-olist-edit-distance">
+          <output-dir compare="Text">inverted-index-olist-edit-distance</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="inverted-index-olist-edit-distance-panic">
+          <output-dir compare="Text">inverted-index-olist-edit-distance-panic</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="inverted-index-olist-jaccard">
+          <output-dir compare="Text">inverted-index-olist-jaccard</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="inverted-index-ulist-jaccard">
+          <output-dir compare="Text">inverted-index-ulist-jaccard</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="inverted-index-word-contains">
+          <output-dir compare="Text">inverted-index-word-contains</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="inverted-index-word-jaccard">
+          <output-dir compare="Text">inverted-index-word-jaccard</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="orders-index-custkey">
+          <output-dir compare="Text">orders-index-custkey</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="orders-index-custkey-conjunctive">
+          <output-dir compare="Text">orders-index-custkey-conjunctive</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="orders-index-custkey-conjunctive-open">
+          <output-dir compare="Text">orders-index-custkey-conjunctive-open</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="orders-index-custkey-open">
+          <output-dir compare="Text">orders-index-custkey-open</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="range-search">
+          <output-dir compare="Text">range-search</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="range-search-open">
+          <output-dir compare="Text">range-search-open</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="rtree-secondary-index">
+          <output-dir compare="Text">rtree-secondary-index</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="rtree-secondary-index-nullable">
+          <output-dir compare="Text">rtree-secondary-index-nullable</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="nested-index/index-selection">
+        <compilation-unit name="rtree-secondary-index-open">
+          <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>
+  <test-group name="nested-index-dml">
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="compact-dataset-and-its-indexes">
+        <output-dir compare="Text">compact-dataset-and-its-indexes</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="nested-uuid-load">
+        <output-dir compare="Text">nested-uuid-load</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="nested-uuid-insert">
+        <output-dir compare="Text">nested-uuid-insert</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="delete-from-loaded-dataset-with-index">
+        <output-dir compare="Text">delete-from-loaded-dataset-with-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="delete-from-loaded-dataset-with-sec-primary-index">
+        <output-dir compare="Text">delete-from-loaded-dataset-with-sec-primary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="drop-index">
+        <output-dir compare="Text">drop-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--test-case FilePath="nested-index-dml">
+      <compilation-unit name="insert-into-empty-dataset-with-index">
+        <output-dir compare="Text">insert-into-empty-dataset-with-index</output-dir>
+      </compilation-unit>
+    </test-case!-->
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="insert-into-empty-dataset-with-sec-primary-index">
+        <output-dir compare="Text">insert-into-empty-dataset-with-sec-primary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--test-case FilePath="nested-index-dml">
+      <compilation-unit name="insert-into-loaded-dataset-with-index_01">
+        <output-dir compare="Text">insert-into-loaded-dataset-with-index_01</output-dir>
+      </compilation-unit>
+    </test-case!-->
+    <!--test-case FilePath="nested-index-dml">
+      <compilation-unit name="insert-into-loaded-dataset-with-index_02">
+        <output-dir compare="Text">insert-into-loaded-dataset-with-index_02</output-dir>
+      </compilation-unit>
+    </test-case!-->
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="insert-into-loaded-dataset-with-sec-primary-index">
+        <output-dir compare="Text">insert-into-loaded-dataset-with-sec-primary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--test-case FilePath="nested-index-dml">
+      <compilation-unit name="load-with-index">
+        <output-dir compare="Text">load-with-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="load-with-sec-primary-index">
+        <output-dir compare="Text">load-with-sec-primary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="load-with-ngram-index">
+        <output-dir compare="Text">load-with-ngram-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="load-with-rtree-index">
+        <output-dir compare="Text">load-with-rtree-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="load-with-word-index">
+        <output-dir compare="Text">load-with-word-index</output-dir>
+      </compilation-unit>
+    </test-case!-->
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-delete-btree-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-btree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-delete-rtree-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-rtree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-delete-rtree-secondary-index">
+        <output-dir compare="Text">scan-delete-rtree-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-insert-btree-secondary-index-nullable">
+        <output-dir compare="Text">scan-insert-btree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-insert-rtree-secondary-index-nullable">
+        <output-dir compare="Text">scan-insert-rtree-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-insert-rtree-secondary-index">
+        <output-dir compare="Text">scan-insert-rtree-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index">
+        <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-insert-inverted-index-word-secondary-index">
+        <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index-nullable">
+        <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-insert-inverted-index-word-secondary-index-nullable">
+        <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index">
+        <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-delete-inverted-index-word-secondary-index">
+        <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nested-index-dml">
+      <compilation-unit name="scan-delete-inverted-index-word-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="array-index">
+    <test-group name="array-index/error-handling">
+      <test-case FilePath="array-index/error-handling">
+        <compilation-unit name="index-include-unknown-key">
+          <output-dir compare="Text">index-include-unknown-key</output-dir>
+          <expected-error>ASX1079: Compilation error: Array indexes must specify EXCLUDE UNKNOWN KEY</expected-error>
+          <expected-error>ASX1079: Compilation error: Array indexes must specify EXCLUDE UNKNOWN KEY</expected-error>
+          <source-location>false</source-location>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/error-handling">
+        <compilation-unit name="index-two-array-fields">
+          <output-dir compare="Text">index-two-array-fields</output-dir>
+          <expected-error>ASX1079: Compilation error: Cannot create composite index with multiple array fields using different arrays</expected-error>
+          <expected-error>ASX1079: Compilation error: Cannot create composite index with multiple array fields using different arrays</expected-error>
+          <source-location>false</source-location>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/error-handling">
+        <compilation-unit name="invalid-array-path">
+          <output-dir compare="Text">invalid-array-path</output-dir>
+          <expected-error>ASX0037: Type mismatch: expected value of type array or multiset, but got the value of type CheckinType_checkin_time:</expected-error>
+          <expected-error>ASX0037: Type mismatch: expected value of type array or multiset, but got the value of type string</expected-error>
+          <source-location>false</source-location>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/error-handling">
+        <compilation-unit name="index-on-closed-array">
+          <output-dir compare="Text">index-on-closed-array</output-dir>
+          <expected-error>ASX1014: Field 'date' is not found</expected-error>
+          <source-location>false</source-location>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/error-handling">
+        <compilation-unit name="index-with-enforced-type">
+          <output-dir compare="Text">index-with-enforced-type</output-dir>
+          <expected-error>ASX1155: Incompatible index type ARRAY</expected-error>
+          <source-location>false</source-location>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="array-index/metadata">
+      <test-case FilePath="array-index/metadata/closed">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/open">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/closed">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/open">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/closed">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/open">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/closed">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/open">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/open">
+        <compilation-unit name="complex-structures">
+          <output-dir compare="Text">complex-structures</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/closed">
+        <compilation-unit name="with-composite-sk">
+          <output-dir compare="Text">with-composite-sk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/open">
+        <compilation-unit name="with-composite-sk">
+          <output-dir compare="Text">with-composite-sk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/closed">
+        <compilation-unit name="with-composite-array-different-indicators">
+          <output-dir compare="Text">with-composite-array-different-indicators</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/open">
+        <compilation-unit name="with-composite-array-different-indicators">
+          <output-dir compare="Text">with-composite-array-different-indicators</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/closed">
+        <compilation-unit name="with-3-level-record-path">
+          <output-dir compare="Text">with-3-level-record-path</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/metadata/open">
+        <compilation-unit name="with-3-level-record-path">
+          <output-dir compare="Text">with-3-level-record-path</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="array-index/bulk-loading/on-index-creation">
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+        <compilation-unit name="with-composite-pk">
+          <output-dir compare="Text">with-composite-pk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+        <compilation-unit name="with-filter-fields">
+          <output-dir compare="Text">with-filter-fields</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+        <compilation-unit name="with-3-level-record-path">
+          <output-dir compare="Text">with-3-level-record-path</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+        <compilation-unit name="composite-atomic">
+          <output-dir compare="Text">composite-atomic</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="array-index/bulk-loading/after-index-creation">
+      <test-case FilePath="array-index/bulk-loading/after-index-creation">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/after-index-creation">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/after-index-creation">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/after-index-creation">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/after-index-creation">
+        <compilation-unit name="with-3-level-record-path">
+          <output-dir compare="Text">with-3-level-record-path</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/after-index-creation">
+        <compilation-unit name="with-composite-pk">
+          <output-dir compare="Text">with-composite-pk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/after-index-creation">
+        <compilation-unit name="with-filter-fields">
+          <output-dir compare="Text">with-filter-fields</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/bulk-loading/after-index-creation">
+        <compilation-unit name="with-open-index">
+          <output-dir compare="Text">with-open-index</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="array-index/insert-upsert-delete">
+      <test-case FilePath="array-index/insert-upsert-delete/closed">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/open">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/closed">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/open">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/closed">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/open">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/closed">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/open">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/closed">
+        <compilation-unit name="with-composite-sk">
+          <output-dir compare="Text">with-composite-sk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/open">
+        <compilation-unit name="with-composite-sk">
+          <output-dir compare="Text">with-composite-sk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/closed">
+        <compilation-unit name="with-additional-atomic-index">
+          <output-dir compare="Text">with-additional-atomic-index</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/open">
+        <compilation-unit name="with-additional-atomic-index">
+          <output-dir compare="Text">with-additional-atomic-index</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/insert-upsert-delete/closed">
+        <compilation-unit name="with-filter-fields">
+          <output-dir compare="Text">with-filter-fields</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="array-index/join-quantified-queries">
+      <test-case FilePath="array-index/join-quantified-queries">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/join-quantified-queries">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/join-quantified-queries">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/join-quantified-queries">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="array-index/join-unnest-queries">
+      <test-case FilePath="array-index/join-unnest-queries">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/join-unnest-queries">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/join-unnest-queries">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/join-unnest-queries">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/join-unnest-queries">
+        <compilation-unit name="with-open-index">
+          <output-dir compare="Text">with-open-index</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/join-unnest-queries">
+        <compilation-unit name="loj-subquery">
+          <output-dir compare="Text">loj-subquery</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="array-index/select-unnest-queries">
+      <test-case FilePath="array-index/select-unnest-queries/closed">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/open">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/closed">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/open">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/closed">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/open">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/closed">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/open">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/closed">
+        <compilation-unit name="with-3-level-record-path">
+          <output-dir compare="Text">with-3-level-record-path</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/open">
+        <compilation-unit name="with-3-level-record-path">
+          <output-dir compare="Text">with-3-level-record-path</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/closed">
+        <compilation-unit name="with-composite-sk">
+          <output-dir compare="Text">with-composite-sk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/open">
+        <compilation-unit name="with-composite-sk">
+          <output-dir compare="Text">with-composite-sk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/closed">
+        <compilation-unit name="with-composite-pk">
+          <output-dir compare="Text">with-composite-pk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/closed">
+        <compilation-unit name="with-filter-fields">
+          <output-dir compare="Text">with-filter-fields</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/open">
+        <compilation-unit name="using-feed-new-index">
+          <output-dir compare="Text">using-feed</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-unnest-queries/open">
+        <compilation-unit name="using-feed-old-index">
+          <output-dir compare="Text">using-feed</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="array-index/select-quantified-queries">
+      <test-case FilePath="array-index/select-quantified-queries">
+        <compilation-unit name="use-case-1">
+          <output-dir compare="Text">use-case-1</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-quantified-queries">
+        <compilation-unit name="use-case-2">
+          <output-dir compare="Text">use-case-2</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-quantified-queries">
+        <compilation-unit name="use-case-3">
+          <output-dir compare="Text">use-case-3</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-quantified-queries">
+        <compilation-unit name="use-case-4">
+          <output-dir compare="Text">use-case-4</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-quantified-queries">
+        <compilation-unit name="with-composite-pk">
+          <output-dir compare="Text">with-composite-pk</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="array-index/select-quantified-queries">
+        <compilation-unit name="with-open-index">
+          <output-dir compare="Text">with-open-index</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+    <test-group name="array-index/composite-index-queries">
+      <test-case FilePath="array-index">
+        <compilation-unit name="composite-index-queries">
+          <output-dir compare="Text">composite-index-queries</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
+  </test-group>
+  <test-group name="nestrecords">
+    <test-case FilePath="nestrecords">
+      <compilation-unit name="nestrecord">
+        <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-case FilePath="nestrecords">
+      <compilation-unit name="query-ASTERIXDB-1025">
+        <output-dir compare="Text">query-ASTERIXDB-1025</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="null-missing">
+    <test-case FilePath="null-missing">
+      <compilation-unit name="array">
+        <output-dir compare="Text">array</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="boolean">
+        <output-dir compare="Text">boolean</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="ceil">
+        <output-dir compare="Text">ceil</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="comparison">
+        <output-dir compare="Text">comparison</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="field-access">
+        <output-dir compare="Text">field-access</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="group-by">
+        <output-dir compare="Text">group-by</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="group-by-from-dataset">
+        <output-dir compare="Text">group-by-from-dataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="length">
+        <output-dir compare="Text">length</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="lower">
+        <output-dir compare="Text">lower</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="order-by">
+        <output-dir compare="Text">order-by</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="order-by-2">
+        <output-dir compare="Text">order-by-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="order-by-3-negative">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error><![CDATA[ASX1001: Syntax error: In line 33 >>ORDER BY d.a NULLS;<< Encountered ";" at column 19]]></expected-error>
+        <expected-error><![CDATA[ASX1001: Syntax error: Unexpected token: NULLS (in line 33, at column 20)]]></expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="order-by-from-dataset">
+        <output-dir compare="Text">order-by-from-dataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="order-by-from-dataset-2">
+        <output-dir compare="Text">order-by-from-dataset-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="scan-collection">
+        <output-dir compare="Text">scan-collection</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="spatial-intersect">
+        <output-dir compare="Text">spatial-intersect</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="substr">
+        <output-dir compare="Text">substr</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="upper">
+        <output-dir compare="Text">upper</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="is">
+        <output-dir compare="Text">is</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="ifmissing">
+        <output-dir compare="Text">ifmissing</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="ifnull">
+        <output-dir compare="Text">ifnull</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="ifmissingornull">
+        <output-dir compare="Text">ifmissingornull</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="coalesce">
+        <output-dir compare="Text">coalesce</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="query-ASTERIXDB-1689">
+        <output-dir compare="Text">query-ASTERIXDB-1689</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="null-missing">
+      <compilation-unit name="print-ASTERIXDB-1885">
+        <output-dir compare="Text">print-ASTERIXDB-1885</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="numeric">
+    <test-case FilePath="numeric">
+      <compilation-unit name="caret0">
+        <output-dir compare="Text">caret0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="caret1">
+        <output-dir compare="Text">caret1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="abs0">
+        <output-dir compare="Text">abs0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="abs1">
+        <output-dir compare="Text">abs1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="abs2">
+        <output-dir compare="Text">abs2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="abs3">
+        <output-dir compare="Text">abs3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="abs4">
+        <output-dir compare="Text">abs4</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="add_double">
+        <output-dir compare="Text">add_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="add_float">
+        <output-dir compare="Text">add_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="add_int16">
+        <output-dir compare="Text">add_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="add_int32">
+        <output-dir compare="Text">add_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="add_int64">
+        <output-dir compare="Text">add_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="add_int8">
+        <output-dir compare="Text">add_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--
+    <test-case FilePath="numeric">
+      <compilation-unit name="issue_1166">
+        <output-dir compare="Text">issue_1166</output-dir>
+      </compilation-unit>
+    </test-case>
+    -->
+    <test-case FilePath="numeric">
+      <compilation-unit name="ceiling0">
+        <output-dir compare="Text">ceiling0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="ceiling1">
+        <output-dir compare="Text">ceiling1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="ceiling2">
+        <output-dir compare="Text">ceiling2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="ceiling3">
+        <output-dir compare="Text">ceiling3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="ceiling4">
+        <output-dir compare="Text">ceiling4</output-dir>
+      </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>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="divide_float">
+        <output-dir compare="Text">divide_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="divide_int16">
+        <output-dir compare="Text">divide_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="divide_int32">
+        <output-dir compare="Text">divide_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="divide_int64">
+        <output-dir compare="Text">divide_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="divide_int8">
+        <output-dir compare="Text">divide_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="div_mod_case_insensitive">
+        <output-dir compare="Text">div_mod_case_insensitive</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="floor0">
+        <output-dir compare="Text">floor0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="floor1">
+        <output-dir compare="Text">floor1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="floor2">
+        <output-dir compare="Text">floor2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="floor3">
+        <output-dir compare="Text">floor3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="floor4">
+        <output-dir compare="Text">floor4</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="ifinf">
+        <output-dir compare="Text">ifinf</output-dir>
+        <expected-error>Invalid number of arguments for function if-inf (in line 25, at column 14)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="ifnan">
+        <output-dir compare="Text">ifnan</output-dir>
+        <expected-error>Invalid number of arguments for function if-nan (in line 25, at column 14)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="ifnanorinf">
+        <output-dir compare="Text">ifnanorinf</output-dir>
+        <expected-error>Invalid number of arguments for function if-nan-or-inf (in line 25, at column 14)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="infinity">
+        <output-dir compare="Text">infinity</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="multiply_double">
+        <output-dir compare="Text">multiply_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="multiply_float">
+        <output-dir compare="Text">multiply_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="multiply_int16">
+        <output-dir compare="Text">multiply_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="multiply_int32">
+        <output-dir compare="Text">multiply_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="multiply_int64">
+        <output-dir compare="Text">multiply_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="multiply_int8">
+        <output-dir compare="Text">multiply_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="power">
+        <output-dir compare="Text">power</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="query-ASTERIXDB-2530">
+        <output-dir compare="Text">query-ASTERIXDB-2530</output-dir>
+      </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>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round-half-to-even1">
+        <output-dir compare="Text">round-half-to-even1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round-half-to-even2">
+        <output-dir compare="Text">round-half-to-even2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round-half-to-even20">
+        <output-dir compare="Text">round-half-to-even20</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round-half-to-even21">
+        <output-dir compare="Text">round-half-to-even21</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round-half-to-even22">
+        <output-dir compare="Text">round-half-to-even22</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round-half-to-even23">
+        <output-dir compare="Text">round-half-to-even23</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round-half-to-even24">
+        <output-dir compare="Text">round-half-to-even24</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round-half-to-even3">
+        <output-dir compare="Text">round-half-to-even3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round-half-to-even4">
+        <output-dir compare="Text">round-half-to-even4</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round-half-to-even5">
+        <output-dir compare="Text">round-half-to-even5</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round-half-up">
+        <output-dir compare="Text">round-half-up</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round0">
+        <output-dir compare="Text">round0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round1">
+        <output-dir compare="Text">round1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round2">
+        <output-dir compare="Text">round2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round3">
+        <output-dir compare="Text">round3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round4">
+        <output-dir compare="Text">round4</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round5_with_digit_int8">
+        <output-dir compare="Text">round5_with_digit_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round6_with_digit_int16">
+        <output-dir compare="Text">round6_with_digit_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round7_with_digit_int32">
+        <output-dir compare="Text">round7_with_digit_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round8_with_digit_int64">
+        <output-dir compare="Text">round8_with_digit_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round9_with_digit_float">
+        <output-dir compare="Text">round9_with_digit_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round10_with_digit_double">
+        <output-dir compare="Text">round10_with_digit_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="round11_invalid">
+        <output-dir compare="Text">round11_invalid</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="subtract_double">
+        <output-dir compare="Text">subtract_double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="subtract_float">
+        <output-dir compare="Text">subtract_float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="subtract_int16">
+        <output-dir compare="Text">subtract_int16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="subtract_int32">
+        <output-dir compare="Text">subtract_int32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="subtract_int64">
+        <output-dir compare="Text">subtract_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="subtract_int8">
+        <output-dir compare="Text">subtract_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="trunc">
+        <output-dir compare="Text">trunc</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="unary-minus_double_02">
+        <output-dir compare="Text">unary-minus_double_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="unary-minus_float_02">
+        <output-dir compare="Text">unary-minus_float_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="unary-minus_int_02">
+        <output-dir compare="Text">unary-minus_int_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="unary-minus_null">
+        <output-dir compare="Text">unary-minus_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="query-issue355"> <!-- @Yingyi, This one fails in the aql tests!! -->
+        <output-dir compare="Text">query-issue355</output-dir>
+        <!-- <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error> -->
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="acos">
+        <output-dir compare="Text">acos</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="asin">
+        <output-dir compare="Text">asin</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="atan">
+        <output-dir compare="Text">atan</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="atan2">
+        <output-dir compare="Text">atan2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="cos">
+        <output-dir compare="Text">cos</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="cosh">
+        <output-dir compare="Text">cosh</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="sin">
+        <output-dir compare="Text">sin</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="sinh">
+        <output-dir compare="Text">sinh</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="tan">
+        <output-dir compare="Text">tan</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="tanh">
+        <output-dir compare="Text">tanh</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="exp">
+        <output-dir compare="Text">exp</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="ln">
+        <output-dir compare="Text">ln</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="log">
+        <output-dir compare="Text">log</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="sqrt">
+        <output-dir compare="Text">sqrt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="scientific">
+        <output-dir compare="Text">scientific</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="scientific_error">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier e (in line 24, at column 10)</expected-error>
+        <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier e (in line 24, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="sign">
+        <output-dir compare="Text">sign</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="non-finite">
+        <output-dir compare="Clean-JSON">non-finite</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="add_int8">
+        <output-dir compare="Text">add_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="multiply_int8">
+        <output-dir compare="Text">multiply_int8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="numeric">
+      <compilation-unit name="unary-minus_double_02">
+        <output-dir compare="Text">unary-minus_double_02</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="open-closed">
+    <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="c2c-w-optional">
+            <output-dir compare="Text">c2c-w-optional</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="c2c-wo-optional">
+            <output-dir compare="Text">c2c-wo-optional</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="c2c">
+            <output-dir compare="Text">c2c</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <test-case FilePath="open-closed">
+      <compilation-unit name="heterog-list-ordered01">
+        <output-dir compare="Text">heterog-list-ordered01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="heterog-list01">
+        <output-dir compare="Text">heterog-list01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="heterog-list02">
+            <output-dir compare="Text">heterog-list02</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="heterog-list03">
+            <output-dir compare="Text">heterog-list03</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <test-case FilePath="open-closed">
+      <compilation-unit name="open-closed-01">
+        <output-dir compare="Text">open-closed-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="open-closed-12">
+        <output-dir compare="Text">open-closed-12</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="open-closed-14">
+        <output-dir compare="Text">open-closed-14</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue134">
+        <output-dir compare="Text">query-issue134</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue55">
+        <output-dir compare="Text">query-issue55</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue55-1">
+        <output-dir compare="Text">query-issue55-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue166">
+        <output-dir compare="Text">query-issue166</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue208">
+        <output-dir compare="Text">query-issue208</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue236">
+        <output-dir compare="Text">query-issue236</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-15">
+            <output-dir compare="Text">open-closed-15</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-16">
+            <output-dir compare="Text">open-closed-16</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-17">
+            <output-dir compare="Text">open-closed-17</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-19">
+            <output-dir compare="Text">open-closed-19</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-20">
+            <output-dir compare="Text">open-closed-20</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-21">
+            <output-dir compare="Text">open-closed-21</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-22">
+            <output-dir compare="Text">open-closed-22</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <test-case FilePath="open-closed">
+      <compilation-unit name="open-closed-24">
+        <output-dir compare="Text">open-closed-24</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="open-closed-25">
+        <output-dir compare="Text">open-closed-25</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="open-closed-26">
+        <output-dir compare="Text">open-closed-26</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-28">
+            <output-dir compare="Text">open-closed-28</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <test-case FilePath="open-closed">
+      <compilation-unit name="open-closed-29">
+        <output-dir compare="Text">open-closed-29</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--
+        <test-case FilePath="open-closed">
+          <compilation-unit name="open-closed-30">
+            <output-dir compare="Text">open-closed-30</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <test-case FilePath="open-closed">
+      <compilation-unit name="open-closed-31">
+        <output-dir compare="Text">open-closed-31</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="open-closed-32">
+        <output-dir compare="Text">open-closed-32</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="open-closed-33">
+        <output-dir compare="Text">open-closed-33</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-proposal02">
+        <output-dir compare="Text">query-proposal02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-proposal">
+        <output-dir compare="Text">query-proposal</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue350">
+        <output-dir compare="Text">query-issue350</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue350-2">
+        <output-dir compare="Text">query-issue350-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue343">
+        <output-dir compare="Text">query-issue343</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue343-2">
+        <output-dir compare="Text">query-issue343-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue196">
+        <output-dir compare="Text">query-issue196</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue377">
+        <output-dir compare="Text">query-issue377</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue410">
+        <output-dir compare="Text">query-issue410</output-dir>
+        <expected-error>Field type double cannot be promoted to type string</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue453">
+        <output-dir compare="Text">query-issue453</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue453-2">
+        <output-dir compare="Text">query-issue453-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue456">
+        <output-dir compare="Text">query-issue456</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue465">
+        <output-dir compare="Text">query-issue465</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue258">
+        <output-dir compare="Text">query-issue258</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue423">
+        <output-dir compare="Text">query-issue423</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue423-2">
+        <output-dir compare="Text">query-issue423-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue442"><!-- Exception is not thrown!! -->
+        <output-dir compare="Text">query-issue442</output-dir>
+        <!-- <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error>  -->
+      </compilation-unit>
+    </test-case>
+    <!--<test-case FilePath="open-closed">
+            <compilation-unit name="query-issue487">
+                <output-dir compare="Text">query-issue487</output-dir>
+                <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error>
+            </compilation-unit>
+        </test-case> -->
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue592">
+        <output-dir compare="Text">query-issue592</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue625">
+        <output-dir compare="Text">query-issue625</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue208">
+        <output-dir compare="Text">query-issue208</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue236">
+        <output-dir compare="Text">query-issue236</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="quantifiers">
+    <test-case FilePath="quantifiers">
+      <compilation-unit name="any_and_every_01">
+        <output-dir compare="Text">any_and_every_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="quantifiers">
+      <compilation-unit name="anysat_01">
+        <output-dir compare="Text">somesat_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="quantifiers">
+      <compilation-unit name="everysat_01">
+        <output-dir compare="Text">everysat_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="quantifiers">
+      <compilation-unit name="query-ASTERIXDB-1005">
+        <output-dir compare="Text">query-ASTERIXDB-1005</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="quantifiers">
+      <compilation-unit name="query-ASTERIXDB-1674">
+        <output-dir compare="Text">query-ASTERIXDB-1674</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="quantifiers">
+      <compilation-unit name="query-ASTERIXDB-2307">
+        <output-dir compare="Text">query-ASTERIXDB-2307</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="quantifiers">
+      <compilation-unit name="query-ASTERIXDB-2696">
+        <output-dir compare="Text">query-ASTERIXDB-2696</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="quantifiers">
+      <compilation-unit name="query-ASTERIXDB-2947">
+        <output-dir compare="Text">query-ASTERIXDB-2947</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--
+        <test-case FilePath="quantifiers">
+          <compilation-unit name="everysat_02">
+            <output-dir compare="Text">everysat_02</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <!--
+        <test-case FilePath="quantifiers">
+          <compilation-unit name="everysat_03">
+            <output-dir compare="Text">everysat_03</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <test-case FilePath="quantifiers">
+      <compilation-unit name="everysat_04">
+        <output-dir compare="Text">everysat_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="quantifiers">
+      <compilation-unit name="somesat_01">
+        <output-dir compare="Text">somesat_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="quantifiers">
+      <compilation-unit name="somesat_02">
+        <output-dir compare="Text">somesat_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--
+        <test-case FilePath="quantifiers">
+          <compilation-unit name="somesat_03">
+            <output-dir compare="Text">somesat_03</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <!--
+        <test-case FilePath="quantifiers">
+          <compilation-unit name="somesat_04">
+            <output-dir compare="Text">somesat_04</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <!--
+        <test-case FilePath="quantifiers">
+          <compilation-unit name="somesat_05">
+            <output-dir compare="Text">somesat_05</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <test-case FilePath="quantifiers">
+      <compilation-unit name="somesat_06">
+        <output-dir compare="Text">somesat_06</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="range-hints">
+    <test-case FilePath="range-hints">
+      <compilation-unit name="order-by">
+        <output-dir compare="Text">order-by</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!-- Fail sporadically        <test-case FilePath="range-hints">
+            <compilation-unit name="order-by-exception_01">
+                <output-dir compare="Text">order-by</output-dir>
+                <expected-error>org.json.JSONException: JSONObject['summary'] not found</expected-error>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="range-hints">
+            <compilation-unit name="order-by-exception_02">
+                <output-dir compare="Text">order-by</output-dir>
+                <expected-error>org.json.JSONException: JSONObject['summary'] not found</expected-error>
+            </compilation-unit>
+        </test-case> -->
+  </test-group>
+  <test-group name="resolution">
+    <test-case FilePath="resolution">
+      <compilation-unit name="conflict-field-dataset">
+        <output-dir compare="Text">conflict-field-dataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="resolution">
+      <compilation-unit name="conflict-field-dataset-from">
+        <output-dir compare="Text">conflict-field-dataset-from</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="resolution">
+      <compilation-unit name="conflict-field-dataset-fromterm">
+        <output-dir compare="Text">conflict-field-dataset-from</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="resolution">
+      <compilation-unit name="conflict-field-dataset-join">
+        <output-dir compare="Text">conflict-field-dataset-from</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="resolution">
+      <compilation-unit name="conflict-field-dataset-quantifier">
+        <output-dir compare="Text">conflict-field-dataset-from</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="resolution">
+      <compilation-unit name="conflict-field-dataset-unnest">
+        <output-dir compare="Text">conflict-field-dataset-from</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="resolution">
+      <compilation-unit name="conflict-fields-dataset">
+        <output-dir compare="Text">conflict-field-dataset</output-dir>
+        <expected-error>Cannot resolve ambiguous alias reference for identifier samptable</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="resolution">
+      <compilation-unit name="fullyqualified">
+        <output-dir compare="Text">fullyqualified</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="resolution">
+      <compilation-unit name="fullyqualified2">
+        <output-dir compare="Text">fullyqualified2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="resolution">
+      <compilation-unit name="order_1">
+        <output-dir compare="Text">order_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="resolution">
+      <compilation-unit name="groupby_rename_with_sugar">
+        <output-dir compare="Text">groupby_rename_with_sugar</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="resolution">
+      <compilation-unit name="function_dataverse">
+        <output-dir compare="Text">function_dataverse</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="resolution">
+      <compilation-unit name="field_accessor_1">
+        <output-dir compare="Text">field_accessor_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="resolution">
+      <compilation-unit name="field_accessor_2_negative">
+        <output-dir compare="Text">field_accessor_1</output-dir>
+        <expected-error>ASX1074: Cannot resolve ambiguous alias reference for identifier c2 (in line 25, at column 7)</expected-error>
+        <expected-error>ASX1074: Cannot resolve ambiguous alias reference for identifier c2 (in line 25, at column 7)</expected-error>
+        <expected-error>ASX1074: Cannot resolve ambiguous alias reference for identifier c2 (in line 25, at column 7)</expected-error>
+        <expected-error>ASX1074: Cannot resolve ambiguous alias reference for identifier c10 (in line 25, at column 51)</expected-error>
+        <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier c20 (in line 26, at column 8)</expected-error>
+        <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier c1 (in line 25, at column 19)</expected-error>
+        <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier samptable (in line 25, at column 14)</expected-error>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="scan">
+    <test-case FilePath="scan">
+      <compilation-unit name="10">
+        <output-dir compare="Text">10</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="scan">
+      <compilation-unit name="20">
+        <output-dir compare="Text">20</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="scan">
+      <compilation-unit name="issue238_query_1">
+        <output-dir compare="Text">issue238_query_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="scan">
+      <compilation-unit name="issue238_query_2">
+        <output-dir compare="Text">issue238_query_2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!-- The syntax this test case tets isn't really invalid
+        <test-case FilePath="scan">
+            <compilation-unit name="invalid-scan-syntax">
+                <output-dir compare="Text">invalid-scan-syntax</output-dir>
+                <expected-error>java.lang.IllegalStateException: no result file</expected-error>
+            </compilation-unit>
+        </test-case>-->
+    <test-case FilePath="scan">
+      <compilation-unit name="30">
+        <output-dir compare="Text">30</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="scan">
+      <compilation-unit name="alltypes_01">
+        <output-dir compare="Text">alltypes_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="scan">
+      <compilation-unit name="alltypes_01">
+        <output-dir compare="Clean-JSON">alltypes_01-cleanjson</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="scan">
+      <compilation-unit name="alltypes_01">
+        <output-dir compare="Lossless-JSON">alltypes_01-losslessjson</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="scan">
+      <compilation-unit name="alltypes_01">
+        <output-dir compare="Lossless-ADM-JSON">alltypes_01-losslessadmjson</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="scan">
+      <compilation-unit name="alltypes_02">
+        <output-dir compare="Text">alltypes_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="scan">
+      <compilation-unit name="numeric_types_01">
+        <output-dir compare="Text">numeric_types_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="scan">
+      <compilation-unit name="spatial_types_01">
+        <output-dir compare="Text">spatial_types_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--
+        <test-case FilePath="scan">
+          <compilation-unit name="spatial_types_02">
+            <output-dir compare="Text">spatial_types_02</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <test-case FilePath="scan">
+      <compilation-unit name="temp_types_01">
+        <output-dir compare="Text">temp_types_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--
+        <test-case FilePath="scan">
+          <compilation-unit name="temp_types_02">
+            <output-dir compare="Text">temp_types_02</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+  </test-group>
+  <test-group name="select-star">
+    <test-case FilePath="select-star">
+      <compilation-unit name="group_by">
+        <output-dir compare="Text">group_by</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="select-star">
+      <compilation-unit name="from">
+        <output-dir compare="Text">from</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="select-star">
+      <compilation-unit name="join">
+        <output-dir compare="Text">join</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="select-star">
+      <compilation-unit name="mixed">
+        <output-dir compare="Text">mixed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="select-star">
+      <compilation-unit name="no_star">
+        <output-dir compare="Text">no_star</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="select-star">
+      <compilation-unit name="var_star">
+        <output-dir compare="Text">var_star</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="select-star">
+      <compilation-unit name="var_star_2">
+        <output-dir compare="Text">var_star_2</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="semistructured">
+    <test-case FilePath="semistructured">
+      <compilation-unit name="count-nullable">
+        <output-dir compare="Text">count-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="semistructured">
+      <compilation-unit name="cust-filter">
+        <output-dir compare="Text">cust-filter</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="semistructured">
+      <compilation-unit name="has-param1">
+        <output-dir compare="Text">has-param1</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="similarity">
+    <test-case FilePath="similarity">
+      <compilation-unit name="edit-distance-check_ints">
+        <output-dir compare="Text">edit-distance-check_ints</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="edit-distance-check_strings">
+        <output-dir compare="Text">edit-distance-check_strings</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="edit-distance-check_unicode">
+        <output-dir compare="Text">edit-distance-check_unicode</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="edit-distance-list-is-filterable">
+        <output-dir compare="Text">edit-distance-list-is-filterable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="edit-distance-string-is-filterable">
+        <output-dir compare="Text">edit-distance-string-is-filterable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="edit-distance_ints">
+        <output-dir compare="Text">edit-distance_ints</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="edit-distance_strings">
+        <output-dir compare="Text">edit-distance_strings</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="fuzzyeq-edit-distance">
+        <output-dir compare="Text">fuzzyeq-edit-distance</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="fuzzyeq-similarity-jaccard">
+        <output-dir compare="Text">fuzzyeq-similarity-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="prefix-len-jaccard">
+        <output-dir compare="Text">prefix-len-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="similarity-jaccard-check_ints">
+        <output-dir compare="Text">similarity-jaccard-check_ints</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="similarity-jaccard-check_query">
+        <output-dir compare="Text">similarity-jaccard-check_query</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="similarity-jaccard-check_strings">
+        <output-dir compare="Text">similarity-jaccard-check_strings</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="similarity-jaccard-prefix-check">
+        <output-dir compare="Text">similarity-jaccard-prefix-check</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="similarity-jaccard-prefix">
+        <output-dir compare="Text">similarity-jaccard-prefix</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="similarity-jaccard-sorted-check_ints">
+        <output-dir compare="Text">similarity-jaccard-sorted-check_ints</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="similarity-jaccard-sorted-check_query">
+        <output-dir compare="Text">similarity-jaccard-sorted-check_query</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="similarity-jaccard-sorted-check_strings">
+        <output-dir compare="Text">similarity-jaccard-sorted-check_strings</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="similarity-jaccard-sorted_ints">
+        <output-dir compare="Text">similarity-jaccard-sorted_ints</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="similarity-jaccard-sorted_query">
+        <output-dir compare="Text">similarity-jaccard-sorted_query</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="similarity-jaccard-sorted_strings">
+        <output-dir compare="Text">similarity-jaccard-sorted_strings</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="similarity-jaccard_ints">
+        <output-dir compare="Text">similarity-jaccard_ints</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="similarity-jaccard_query">
+        <output-dir compare="Text">similarity-jaccard_query</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="similarity-jaccard_strings">
+        <output-dir compare="Text">similarity-jaccard_strings</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="similarity-jaccard-check_strings_issue628">
+        <output-dir compare="Text">similarity-jaccard-check_strings_issue628</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="spatial">
+    <test-case FilePath="spatial">
+      <compilation-unit name="cell-aggregation-with-filtering">
+        <output-dir compare="Text">cell-aggregation-with-filtering</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="cell-aggregation">
+        <output-dir compare="Text">cell-aggregation</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="circle_accessor">
+        <output-dir compare="Text">circle_accessor</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="circle-intersect-circle">
+        <output-dir compare="Text">circle-intersect-circle</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="create-rtree-index">
+        <output-dir compare="Text">create-rtree-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="distance-between-points">
+        <output-dir compare="Text">distance-between-points</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="hints_spatial_partitioning">
+        <output-dir compare="Text">hints_spatial_partitioning</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="line_accessor">
+        <output-dir compare="Text">line_accessor</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="line-intersect-circle">
+        <output-dir compare="Text">line-intersect-circle</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="line-intersect-line">
+        <output-dir compare="Text">line-intersect-line</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="line-intersect-polygon">
+        <output-dir compare="Text">line-intersect-polygon</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="line-intersect-rectangle">
+        <output-dir compare="Text">line-intersect-rectangle</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="point_accessor">
+        <output-dir compare="Text">point_accessor</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="point-equals-point">
+        <output-dir compare="Text">point-equals-point</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="point-in-circle">
+        <output-dir compare="Text">point-in-circle</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="point-in-polygon">
+        <output-dir compare="Text">point-in-polygon</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="point-in-rectangle">
+        <output-dir compare="Text">point-in-rectangle</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="point-on-line">
+        <output-dir compare="Text">point-on-line</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="polygon_accessor">
+        <output-dir compare="Text">polygon_accessor</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="polygon-intersect-circle">
+        <output-dir compare="Text">polygon-intersect-circle</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="polygon-intersect-polygon">
+        <output-dir compare="Text">polygon-intersect-polygon</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="polygon-intersect-rectangle">
+        <output-dir compare="Text">polygon-intersect-rectangle</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="rectangle_accessor">
+        <output-dir compare="Text">rectangle_accessor</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="rectangle-intersect-circle">
+        <output-dir compare="Text">rectangle-intersect-circle</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="rectangle-intersect-rectangle">
+        <output-dir compare="Text">rectangle-intersect-rectangle</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="spatial-area">
+        <output-dir compare="Text">spatial-area</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="spatial-large-data">
+        <output-dir compare="Text">spatial-large-data</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="spatial_join_dynamic_partitioning">
+        <output-dir compare="Text">spatial_join_dynamic_partitioning</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="spatial_join_static_partitioning">
+        <output-dir compare="Text">spatial_join_static_partitioning</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="spatial_left_outer_join_st_intersects">
+        <output-dir compare="Text">spatial_left_outer_join_st_intersects</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="spatial_join_projection_check">
+        <output-dir compare="Text">spatial_join_projection_check</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="sql-compat">
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="in_non_list_01">
+        <output-dir compare="Text">in_non_list_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="outer_join_01">
+        <output-dir compare="Text">outer_join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="outer_unnest_01">
+        <output-dir compare="Text">outer_unnest_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="select_star_01">
+        <output-dir compare="Text">select_star_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="select_star_02_negative">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1168: Ambiguous projection in SELECT clause (in line 32, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="subquery_coercion_01_scalar">
+        <output-dir compare="Text">subquery_coercion_01_scalar</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="subquery_coercion_02_scalar_negative">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 32, at column 29)</expected-error>
+        <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 32, at column 20)</expected-error>
+        <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 35, at column 4)</expected-error>
+        <expected-error>ASX1169: Unable to do subquery coercion.  (in line 34, at column 3)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="subquery_coercion_03_cmp">
+        <output-dir compare="Text">subquery_coercion_03_cmp</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="subquery_coercion_04_cmp_negative">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 35, at column 26)</expected-error>
+        <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 35, at column 21)</expected-error>
+        <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 37, at column 3)</expected-error>
+        <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 38, at column 3)</expected-error>
+        <expected-error>ASX1169: Unable to do subquery coercion.  (in line 36, at column 3)</expected-error>
+        <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 35, at column 20)</expected-error>
+        <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 36, at column 11)</expected-error>
+        <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 37, at column 11)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="subquery_coercion_05_in">
+        <output-dir compare="Text">subquery_coercion_05_in</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="subquery_coercion_06_in_negative">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 36, at column 27)</expected-error>
+        <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 37, at column 17)</expected-error>
+        <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 38, at column 3)</expected-error>
+        <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 39, at column 3)</expected-error>
+        <expected-error>ASX1169: Unable to do subquery coercion.  (in line 37, at column 3)</expected-error>
+        <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 36, at column 21)</expected-error>
+        <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 37, at column 25)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="subquery_coercion_07_from">
+        <output-dir compare="Text">subquery_coercion_07_from</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="subquery_coercion_08_misc">
+        <output-dir compare="Text">subquery_coercion_08_misc</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="union_all_01">
+        <output-dir compare="Text">union_all_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="union_all_02_negative">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1171: Unable to process UNION clause. Both SELECT and SELECT VALUE are present (in line 29, at column 1)</expected-error>
+        <expected-error>ASX1171: Unable to process UNION clause. Unequal number of input fields (in line 29, at column 1)</expected-error>
+        <expected-error>ASX1171: Unable to process UNION clause. Unsupported projection kind (in line 29, at column 8)</expected-error>
+        <expected-error>ASX1171: Unable to process UNION clause. Unsupported projection kind (in line 30, at column 8)</expected-error>
+        <expected-error>ASX1171: Unable to process UNION clause. Unsupported projection kind (in line 31, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="statement-params">
+    <!--test-case FilePath="statement-params">
+      <compilation-unit name="index_01">
+        <output-dir compare="Text">index_01</output-dir>
+      </compilation-unit>
+    </test-case!-->
+    <test-case FilePath="statement-params">
+      <compilation-unit name="mixed_01">
+        <output-dir compare="Text">mixed_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="named_01">
+        <output-dir compare="Text">named_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="named_02">
+        <output-dir compare="Text">named_01</output-dir>
+        <expected-error>ASX1086: No value for parameter: $p2</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="named_03">
+        <output-dir compare="Text">named_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="named_04">
+        <output-dir compare="Text">named_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="positional_01">
+        <output-dir compare="Text">positional_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="positional_02">
+        <output-dir compare="Text">positional_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="positional_03">
+        <output-dir compare="Text">positional_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="positional_04">
+        <output-dir compare="Text">positional_02</output-dir>
+        <expected-error>ASX1086: No value for parameter: $2</expected-error>
+        <expected-error>ASX1086: No value for parameter: $3</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="positional_05">
+        <output-dir compare="Text">positional_05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="statement-params">
+      <compilation-unit name="query-ASTERIXDB-2413">
+        <output-dir compare="Text">query-ASTERIXDB-2413</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="string">
+    <test-case FilePath="string">
+      <compilation-unit name="codepoint-to-string1">
+        <output-dir compare="Text">codepoint-to-string1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="codepoint-to-string2">
+        <output-dir compare="Text">codepoint-to-string2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="concat/001">
+        <output-dir compare="Text">concat/001</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="concat/002">
+        <output-dir compare="Text">concat/002</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string" check-warnings="true">
+      <compilation-unit name="concat/003">
+        <output-dir compare="Text">concat/003</output-dir>
+        <expected-warn>Type mismatch: function string-concat expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 26, at column 1)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="concat/004">
+        <output-dir compare="Text">concat/004</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="concat/005">
+        <output-dir compare="Text">concat/005</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="concat/006">
+        <output-dir compare="Text">concat/006</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="concat/007">
+        <output-dir compare="Text">concat/007</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="concat/008">
+        <output-dir compare="Text">concat/002</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="concat/concat_pipe">
+        <output-dir compare="Text">concat/002</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="concat/concat_pipe_multi">
+        <output-dir compare="Text">concat/concat_pipe_multi</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="constructor">
+        <output-dir compare="Text">constructor</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="contains_01">
+        <output-dir compare="Text">contains_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="cpttostr01">
+        <output-dir compare="Text">cpttostr01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="cpttostr02">
+        <output-dir compare="Text">cpttostr02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="cpttostr04">
+        <output-dir compare="Text">cpttostr04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="ends-with1">
+        <output-dir compare="Text">ends-with1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="ends-with2">
+        <output-dir compare="Text">ends-with2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="ends-with3">
+        <output-dir compare="Text">ends-with3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="ends-with4">
+        <output-dir compare="Text">ends-with4</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="ends-with5">
+        <output-dir compare="Text">ends-with5</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="escapes01">
+        <output-dir compare="Text">escapes01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="escapes02">
+        <output-dir compare="Text">escapes02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="initcap">
+        <output-dir compare="Text">initcap</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="join/000">
+        <output-dir compare="Text">join/000</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string" check-warnings="true">
+      <compilation-unit name="join/001">
+        <output-dir compare="Text">join/001</output-dir>
+        <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 24, at column 15)</expected-warn>
+        <expected-warn>Type mismatch: function string-join expects its 1st input parameter to be of type array, but the actual input type is bigint (in line 24, at column 15)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="join/002">
+        <output-dir compare="Text">join/002</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string" check-warnings="true">
+      <compilation-unit name="join/003">
+        <output-dir compare="Text">join/003</output-dir>
+        <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 23, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 30, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-join expects its 1st input parameter to be of type array, but the actual input type is bigint (in line 33, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-join expects its 1st input parameter to be of type array, but the actual input type is bigint (in line 26, at column 1)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string" check-warnings="true">
+      <compilation-unit name="join/004">
+        <output-dir compare="Text">join/004</output-dir>
+        <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 26, at column 1)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string" check-warnings="true">
+      <compilation-unit name="join/005">
+        <output-dir compare="Text">join/005</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="length_01">
+        <output-dir compare="Text">length_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="length_02">
+        <output-dir compare="Text">length_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="length_multi_code_point_01">
+        <output-dir compare="Text">length_multi_code_point_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="like_01">
+        <output-dir compare="Text">like_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="like_02">
+        <output-dir compare="Text">like_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="like_03_negative">
+        <output-dir compare="Text">like_03_negative</output-dir>
+        <expected-error>Invalid pattern '__\c' for LIKE (in line 21, at column 11)</expected-error>
+        <expected-error>Invalid pattern '%\' for LIKE (in line 21, at column 18)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="like_null">
+        <output-dir compare="Text">like_null</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="lowercase">
+        <output-dir compare="Text">lowercase</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="matches02">
+        <output-dir compare="Text">matches02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="matches03">
+        <output-dir compare="Text">matches03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="matches04">
+        <output-dir compare="Text">matches04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="matches05">
+        <output-dir compare="Text">matches05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="matches06">
+        <output-dir compare="Text">matches06</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="matches1">
+        <output-dir compare="Text">matches1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="matches11">
+        <output-dir compare="Text">matches11</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="matches2">
+        <output-dir compare="Text">matches2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="matches21">
+        <output-dir compare="Text">matches21</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="matches22">
+        <output-dir compare="Text">matches22</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="matches23">
+        <output-dir compare="Text">matches23</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="matches3">
+        <output-dir compare="Text">matches3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="matchesnull">
+        <output-dir compare="Text">matchesnull</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="position/offset0/position">
+        <output-dir compare="Text">position/offset0/position</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="position/offset0/pos">
+        <output-dir compare="Text">position/offset0/pos</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="position/offset0/position0">
+        <output-dir compare="Text">position/offset0/position0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="position/offset0/pos0">
+        <output-dir compare="Text">position/offset0/pos0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="position/offset0/pos0_multi_code_point">
+        <output-dir compare="Text">position/offset0/pos0_multi_code_point</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="position/offset1/position1">
+        <output-dir compare="Text">position/offset1/position1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="position/offset1/pos1">
+        <output-dir compare="Text">position/offset1/pos1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="position/offset1/pos1_multi_code_point">
+        <output-dir compare="Text">position/offset1/pos1_multi_code_point</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_contains/regexp_contains">
+        <output-dir compare="Text">regexp_contains/regexp_contains</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_contains/regex_contains">
+        <output-dir compare="Text">regexp_contains/regex_contains</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_contains/contains_regexp">
+        <output-dir compare="Text">regexp_contains/contains_regexp</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_contains/contains_regex">
+        <output-dir compare="Text">regexp_contains/contains_regex</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_contains_with_flag/regexp_contains_with_flag">
+        <output-dir compare="Text">regexp_contains_with_flag/regexp_contains_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_contains_with_flag/regex_contains_with_flag">
+        <output-dir compare="Text">regexp_contains_with_flag/regex_contains_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_contains_with_flag/contains_regexp_with_flag">
+        <output-dir compare="Text">regexp_contains_with_flag/contains_regexp_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_contains_with_flag/contains_regex_with_flag">
+        <output-dir compare="Text">regexp_contains_with_flag/contains_regex_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_like/regexp_like">
+        <output-dir compare="Text">regexp_like/regexp_like</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_like/regex_like">
+        <output-dir compare="Text">regexp_like/regex_like</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_like_with_flag/regexp_like_with_flag">
+        <output-dir compare="Text">regexp_like_with_flag/regexp_like_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_like_with_flag/regex_like_with_flag">
+        <output-dir compare="Text">regexp_like_with_flag/regex_like_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_matches/001">
+        <output-dir compare="Text">regexp_matches/001</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_matches/002">
+        <output-dir compare="Text">regexp_matches/002</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_matches/003">
+        <output-dir compare="Text">regexp_matches/003</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_matches/004">
+        <output-dir compare="Text">regexp_matches/004</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position/offset0/regexp_position">
+        <output-dir compare="Text">regexp_position/offset0/regexp_position</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position/offset0/regexp_pos">
+        <output-dir compare="Text">regexp_position/offset0/regexp_pos</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position/offset0/regexp_position0">
+        <output-dir compare="Text">regexp_position/offset0/regexp_position0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position/offset0/regexp_pos0">
+        <output-dir compare="Text">regexp_position/offset0/regexp_pos0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position/offset0/regex_position">
+        <output-dir compare="Text">regexp_position/offset0/regex_position</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position/offset0/regex_pos">
+        <output-dir compare="Text">regexp_position/offset0/regex_pos</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position/offset0/regex_position0">
+        <output-dir compare="Text">regexp_position/offset0/regex_position0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position/offset0/regex_pos0">
+        <output-dir compare="Text">regexp_position/offset0/regex_pos0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position/offset1/regexp_position1">
+        <output-dir compare="Text">regexp_position/offset1/regexp_position1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position/offset1/regexp_pos1">
+        <output-dir compare="Text">regexp_position/offset1/regexp_pos1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position/offset1/regex_position1">
+        <output-dir compare="Text">regexp_position/offset1/regex_position1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position/offset1/regex_pos1">
+        <output-dir compare="Text">regexp_position/offset1/regex_pos1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position_with_flag/offset0/regexp_position_with_flag">
+        <output-dir compare="Text">regexp_position_with_flag/offset0/regexp_position_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position_with_flag/offset0/regexp_pos_with_flag">
+        <output-dir compare="Text">regexp_position_with_flag/offset0/regexp_pos_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position_with_flag/offset0/regexp_position0_with_flag">
+        <output-dir compare="Text">regexp_position_with_flag/offset0/regexp_position0_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position_with_flag/offset0/regexp_pos0_with_flag">
+        <output-dir compare="Text">regexp_position_with_flag/offset0/regexp_pos0_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position_with_flag/offset0/regex_position_with_flag">
+        <output-dir compare="Text">regexp_position_with_flag/offset0/regex_position_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position_with_flag/offset0/regex_pos_with_flag">
+        <output-dir compare="Text">regexp_position_with_flag/offset0/regex_pos_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substr-ASTERIXDB-2949">
+        <output-dir compare="Text">substr-ASTERIXDB-2949</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position_with_flag/offset0/regex_position0_with_flag">
+        <output-dir compare="Text">regexp_position_with_flag/offset0/regex_position0_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position_with_flag/offset0/regex_pos0_with_flag">
+        <output-dir compare="Text">regexp_position_with_flag/offset0/regex_pos0_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position_with_flag/offset1/regexp_position1_with_flag">
+        <output-dir compare="Text">regexp_position_with_flag/offset1/regexp_position1_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position_with_flag/offset1/regexp_pos1_with_flag">
+        <output-dir compare="Text">regexp_position_with_flag/offset1/regexp_pos1_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position_with_flag/offset1/regex_position1_with_flag">
+        <output-dir compare="Text">regexp_position_with_flag/offset1/regex_position1_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_position_with_flag/offset1/regex_pos1_with_flag">
+        <output-dir compare="Text">regexp_position_with_flag/offset1/regex_pos1_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_split/001">
+        <output-dir compare="Text">regexp_split/001</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_split/002">
+        <output-dir compare="Text">regexp_split/002</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_split/003">
+        <output-dir compare="Text">regexp_split/003</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_split/004">
+        <output-dir compare="Text">regexp_split/004</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="repeat">
+        <output-dir compare="Text">repeat</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="replace">
+        <output-dir compare="Text">replace</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="replace_with_limit">
+        <output-dir compare="Text">replace_with_limit</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_replace/regexp_replace">
+        <output-dir compare="Text">regexp_replace/regexp_replace</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_replace/regex_replace">
+        <output-dir compare="Text">regexp_replace/regex_replace</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_replace_with_flag/regexp_replace_with_flag">
+        <output-dir compare="Text">regexp_replace_with_flag/regexp_replace_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="regexp_replace_with_flag/regex_replace_with_flag">
+        <output-dir compare="Text">regexp_replace_with_flag/regex_replace_with_flag</output-dir>
+      </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="split">
+        <output-dir compare="Text">split</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="starts-with1">
+        <output-dir compare="Text">starts-with1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="starts-with2">
+        <output-dir compare="Text">starts-with2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="starts-with3">
+        <output-dir compare="Text">starts-with3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="starts-with4">
+        <output-dir compare="Text">starts-with4</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="starts-with5">
+        <output-dir compare="Text">starts-with5</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="string-equal1">
+        <output-dir compare="Text">string-equal1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="string-equal2">
+        <output-dir compare="Text">string-equal2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="string-equal3">
+        <output-dir compare="Text">string-equal3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="string-equal4">
+        <output-dir compare="Text">string-equal4</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="string-literal1">
+        <output-dir compare="Text">string-literal1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="string-to-codepoint">
+        <output-dir compare="Text">string-to-codepoint</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="string-to-codepoint1">
+        <output-dir compare="Text">string-to-codepoint1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="string-to-codepoint2">
+        <output-dir compare="Text">string-to-codepoint2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="string_to_codepoint_multi_codepoints_01">
+        <output-dir compare="Text">string_to_codepoint_multi_codepoints_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="strlen02">
+        <output-dir compare="Text">strlen02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="strlen03">
+        <output-dir compare="Text">strlen03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="strtocpt01">
+        <output-dir compare="Text">strtocpt01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="strtocpt02">
+        <output-dir compare="Text">strtocpt02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="strtocpt03">
+        <output-dir compare="Text">strtocpt03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substr01/offset0/substring">
+        <output-dir compare="Text">substr01/offset0/substring</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substr01/offset0/substr">
+        <output-dir compare="Text">substr01/offset0/substr</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substr01/offset0/substring0">
+        <output-dir compare="Text">substr01/offset0/substring0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substr01/offset0/substr0">
+        <output-dir compare="Text">substr01/offset0/substr0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substr01/offset1/substring1">
+        <output-dir compare="Text">substr01/offset1/substring1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substr01/offset1/substr1">
+        <output-dir compare="Text">substr01/offset1/substr1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substr04">
+        <output-dir compare="Text">substr04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substr05">
+        <output-dir compare="Text">substr05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substr06">
+        <output-dir compare="Text">substr06</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring_multi_codepoint_01">
+        <output-dir compare="Text">substring_multi_codepoint_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring-after-1">
+        <output-dir compare="Text">substring-after-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring-after-2">
+        <output-dir compare="Text">substring-after-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring-after-3">
+        <output-dir compare="Text">substring-after-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring-after-4">
+        <output-dir compare="Text">substring-after-4</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring-before-1">
+        <output-dir compare="Text">substring-before-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring-before-2">
+        <output-dir compare="Text">substring-before-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring-before-3">
+        <output-dir compare="Text">substring-before-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring2-1/offset0/substring">
+        <output-dir compare="Text">substring2-1/offset0/substring</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring2-1/offset0/substr">
+        <output-dir compare="Text">substring2-1/offset0/substr</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring2-1/offset0/substring0">
+        <output-dir compare="Text">substring2-1/offset0/substring0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring2-1/offset0/substr0">
+        <output-dir compare="Text">substring2-1/offset0/substr0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring2-1/offset1/substring1">
+        <output-dir compare="Text">substring2-1/offset1/substring1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring2-1/offset1/substr1">
+        <output-dir compare="Text">substring2-1/offset1/substr1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring2-2">
+        <output-dir compare="Text">substring2-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring2-3">
+        <output-dir compare="Text">substring2-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring2-4">
+        <output-dir compare="Text">substring2-4</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring_01">
+        <output-dir compare="Text">substring_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="title">
+        <output-dir compare="Text">initcap</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="toLowerCase02">
+        <output-dir compare="Text">toLowerCase02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="toLowerCase03">
+        <output-dir compare="Text">toLowerCase03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="toLowerCase04">
+        <output-dir compare="Text">toLowerCase04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="trim">
+        <output-dir compare="Text">trim</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="ltrim">
+        <output-dir compare="Text">ltrim</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="rtrim">
+        <output-dir compare="Text">rtrim</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="uppercase">
+        <output-dir compare="Text">uppercase</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="varlen-encoding">
+        <output-dir compare="Text">varlen-encoding</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="utf8">
+        <output-dir compare="Text">utf8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="query-ASTERIXDB-1190">
+        <output-dir compare="Text">query-ASTERIXDB-1190</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="string-equal-public">
+        <output-dir compare="Text">string-equal-public</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring-after-5">
+        <output-dir compare="Text">substring-after-5</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="string">
+      <compilation-unit name="substring-after-6">
+        <output-dir compare="Text">substring-after-5</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="subquery">
+    <test-case FilePath="subquery">
+      <compilation-unit name="aggregate_join">
+        <output-dir compare="Text">aggregate_join</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="aggregate_join_external">
+        <output-dir compare="Text">aggregate_join_external</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="gby_inline">
+        <output-dir compare="Text">gby_inline</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="query-ASTERIXDB-1168">
+        <output-dir compare="Text">query-ASTERIXDB-1168</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="decorrelate_with_unique_id">
+        <output-dir compare="Text">decorrelate_with_unique_id</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="exists">
+        <output-dir compare="Text">exists</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="in">
+        <output-dir compare="Text">in</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="in_as_or">
+        <output-dir compare="Text">in_as_or</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="in_correlated">
+        <output-dir compare="Text">in</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="in_let">
+        <output-dir compare="Text">in_let</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="not_exists">
+        <output-dir compare="Text">not_exists</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="not_in">
+        <output-dir compare="Text">not_in</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="alias_negative">
+        <output-dir compare="Text">alias_negative</output-dir>
+        <expected-error>Need an alias for the enclosed expression</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="relational_division">
+        <output-dir compare="Text">relational_division</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="relational_division2">
+        <output-dir compare="Text">relational_division</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="relational_division3">
+        <output-dir compare="Text">relational_division</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="non_unary_subplan_01">
+        <output-dir compare="Text">non_unary_subplan_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="query-ASTERIXDB-1571">
+        <output-dir compare="Text">query-ASTERIXDB-1571</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="query-ASTERIXDB-1571-2">
+        <output-dir compare="Text">query-ASTERIXDB-1571</output-dir>
+        <expected-error>Need an alias for the enclosed expression</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="query-ASTERIXDB-1571-3">
+        <output-dir compare="Text">query-ASTERIXDB-1571-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="query-ASTERIXDB-1571-4">
+        <output-dir compare="Text">query-ASTERIXDB-1571-4</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="query-ASTERIXDB-1572">
+        <output-dir compare="Text">query-ASTERIXDB-1572</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="query-ASTERIXDB-1574">
+        <output-dir compare="Text">query-ASTERIXDB-1574</output-dir>
+        <expected-error>ASX1091: Type mismatch: expected value of type multiset or array, but got the value of type object (in line 25, at column 41)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="query-ASTERIXDB-1574-2">
+        <output-dir compare="Text">query-ASTERIXDB-1574</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="query-ASTERIXDB-1574-3">
+        <output-dir compare="Text">query-ASTERIXDB-1574</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="query-ASTERIXDB-1597">
+        <output-dir compare="Text">query-ASTERIXDB-1597</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="query-ASTERIXDB-1674">
+        <output-dir compare="Text">query-ASTERIXDB-1674</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="query-ASTERIXDB-2815">
+        <output-dir compare="Text">query-ASTERIXDB-2815</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="query-ASTERIXDB-2845">
+        <output-dir compare="Text">query-ASTERIXDB-2845</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="query-ASTERIXDB-3006">
+        <output-dir compare="Text">query-ASTERIXDB-3006</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="subset-collection">
+    <test-case FilePath="subset-collection">
+      <compilation-unit name="01">
+        <output-dir compare="Text">01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subset-collection">
+      <compilation-unit name="02">
+        <output-dir compare="Text">02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subset-collection">
+      <compilation-unit name="03">
+        <output-dir compare="Text">03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subset-collection">
+      <compilation-unit name="05">
+        <output-dir compare="Text">05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subset-collection">
+      <compilation-unit name="06">
+        <output-dir compare="Text">06</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subset-collection">
+      <compilation-unit name="07">
+        <output-dir compare="Text">07</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="synonym">
+    <test-case FilePath="synonym">
+      <compilation-unit name="synonym-01">
+        <output-dir compare="Text">synonym-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="synonym">
+      <compilation-unit name="synonym-02-negative">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1063: Cannot find dataverse with name UNKNOWN_DATAVERSE</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="synonym">
+      <compilation-unit name="synonym-03">
+        <output-dir compare="Text">synonym-03</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="tokenizers">
+    <test-case FilePath="tokenizers">
+      <compilation-unit name="counthashed-gram-tokens_01">
+        <output-dir compare="Text">counthashed-gram-tokens_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tokenizers">
+      <compilation-unit name="counthashed-gram-tokens_02">
+        <output-dir compare="Text">counthashed-gram-tokens_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tokenizers">
+      <compilation-unit name="counthashed-word-tokens_01">
+        <output-dir compare="Text">counthashed-word-tokens_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tokenizers">
+      <compilation-unit name="gram-tokens_01">
+        <output-dir compare="Text">gram-tokens_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tokenizers">
+      <compilation-unit name="gram-tokens_02">
+        <output-dir compare="Text">gram-tokens_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tokenizers">
+      <compilation-unit name="hashed-gram-tokens_01">
+        <output-dir compare="Text">hashed-gram-tokens_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tokenizers">
+      <compilation-unit name="hashed-gram-tokens_02">
+        <output-dir compare="Text">hashed-gram-tokens_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tokenizers">
+      <compilation-unit name="hashed-word-tokens_01">
+        <output-dir compare="Text">hashed-word-tokens_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tokenizers">
+      <compilation-unit name="word-tokens_01">
+        <output-dir compare="Text">word-tokens_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tokenizers">
+      <compilation-unit name="word-tokens_02">
+        <output-dir compare="Text">word-tokens_02</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="tpcds">
+    <test-case FilePath="tpcds">
+      <compilation-unit name="query-ASTERIXDB-1580">
+        <output-dir compare="Text">query-ASTERIXDB-1580</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="query-ASTERIXDB-1581">
+        <output-dir compare="Text">query-ASTERIXDB-1581</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="query-ASTERIXDB-1581-2">
+        <output-dir compare="Text">query-ASTERIXDB-1581-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="query-ASTERIXDB-1581-3">
+        <output-dir compare="Text">query-ASTERIXDB-1581-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="query-ASTERIXDB-1581-4">
+        <output-dir compare="Text">query-ASTERIXDB-1581-4</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="query-ASTERIXDB-1581-5">
+        <output-dir compare="Text">query-ASTERIXDB-1581-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="query-ASTERIXDB-1581-6">
+        <output-dir compare="Text">query-ASTERIXDB-1581-6</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="query-ASTERIXDB-1581-correlated">
+        <output-dir compare="Text">query-ASTERIXDB-1581-correlated</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="query-ASTERIXDB-1581-correlated-2">
+        <output-dir compare="Text">query-ASTERIXDB-1581-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="query-ASTERIXDB-1591">
+        <output-dir compare="Text">query-ASTERIXDB-1591</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="query-ASTERIXDB-1596">
+        <output-dir compare="Text">query-ASTERIXDB-1596</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="query-ASTERIXDB-1602">
+        <output-dir compare="Text">query-ASTERIXDB-1602</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q01">
+        <output-dir compare="Text">q01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q03">
+        <output-dir compare="Text">q03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q05">
+        <output-dir compare="Text">q05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q07">
+        <output-dir compare="Text">q07</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q09">
+        <output-dir compare="Text">q09</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q12">
+        <output-dir compare="Text">q12</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q15">
+        <output-dir compare="Text">q15</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q18">
+        <output-dir compare="Text">q18</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q19">
+        <output-dir compare="Text">q19</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q20">
+        <output-dir compare="Text">q20</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q21">
+        <output-dir compare="Text">q21</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q22">
+        <output-dir compare="Text">q22</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q24a">
+        <output-dir compare="Text">q24a</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q24b">
+        <output-dir compare="Text">q24b</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q25">
+        <output-dir compare="Text">q25</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q26">
+        <output-dir compare="Text">q26</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q27">
+        <output-dir compare="Text">q27</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q29">
+        <output-dir compare="Text">q29</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q30">
+        <output-dir compare="Text">q30</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q31">
+        <output-dir compare="Text">q31</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q34">
+        <output-dir compare="Text">q34</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q36">
+        <output-dir compare="Text">q36</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q37">
+        <output-dir compare="Text">q37</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q41">
+        <output-dir compare="Text">q41</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q42">
+        <output-dir compare="Text">q42</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q43">
+        <output-dir compare="Text">q43</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q45">
+        <output-dir compare="Text">q45</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q46">
+        <output-dir compare="Text">q46</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q49">
+        <output-dir compare="Text">q49</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q50">
+        <output-dir compare="Text">q50</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q52">
+        <output-dir compare="Text">q52</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q55">
+        <output-dir compare="Text">q55</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q57">
+        <output-dir compare="Text">q57</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q59">
+        <output-dir compare="Text">q59</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q62">
+        <output-dir compare="Text">q62</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q63">
+        <output-dir compare="Text">q63</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q67">
+        <output-dir compare="Text">q67</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q68">
+        <output-dir compare="Text">q68</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q70">
+        <output-dir compare="Text">q70</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q73">
+        <output-dir compare="Text">q73</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q77">
+        <output-dir compare="Text">q77</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q79">
+        <output-dir compare="Text">q79</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q80">
+        <output-dir compare="Text">q80</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q81">
+        <output-dir compare="Text">q81</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q82">
+        <output-dir compare="Text">q82</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q85">
+        <output-dir compare="Text">q85</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q86">
+        <output-dir compare="Text">q86</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q88">
+        <output-dir compare="Text">q88</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q89">
+        <output-dir compare="Text">q89</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q90">
+        <output-dir compare="Text">q90</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q91">
+        <output-dir compare="Text">q91</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q92">
+        <output-dir compare="Text">q92</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q94">
+        <output-dir compare="Text">q94</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q95">
+        <output-dir compare="Text">q95</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q96">
+        <output-dir compare="Text">q96</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpcds">
+      <compilation-unit name="q98">
+        <output-dir compare="Text">q98</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="tpch">
+    <test-case FilePath="tpch">
+      <compilation-unit name="distinct_by">
+        <output-dir compare="Text">distinct_by</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="group_no_agg">
+        <output-dir compare="Text">group_no_agg</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="nest_aggregate">
+        <output-dir compare="Text">nest_aggregate</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="nest_aggregate2">
+        <output-dir compare="Text">nest_aggregate2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="query-issue638">
+        <output-dir compare="Text">query-issue638</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="query-issue785">
+        <output-dir compare="Text">query-issue785</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="query-issue785-2">
+        <output-dir compare="Text">query-issue785-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="query-issue786">
+        <output-dir compare="Text">query-issue786</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="query-issue601">
+        <output-dir compare="Text">query-issue601</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q10_returned_item">
+        <output-dir compare="Text">q10_returned_item</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q10_returned_item_int64">
+        <output-dir compare="Text">q10_returned_item_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q11_important_stock">
+        <output-dir compare="Text">q11_important_stock</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q12_shipping">
+        <output-dir compare="Text">q12_shipping</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q13_customer_distribution">
+        <output-dir compare="Text">q13_customer_distribution</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q14_promotion_effect">
+        <output-dir compare="Text">q14_promotion_effect</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q15_top_supplier">
+        <output-dir compare="Text">q15_top_supplier</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q16_parts_supplier_relationship">
+        <output-dir compare="Text">q16_parts_supplier_relationship</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q17_small_quantity_order_revenue">
+        <output-dir compare="Text">q17_small_quantity_order_revenue</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q17_large_gby_variant">
+        <output-dir compare="Text">q17_large_gby_variant</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q18_large_volume_customer">
+        <output-dir compare="Text">q18_large_volume_customer</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q19_discounted_revenue">
+        <output-dir compare="Text">q19_discounted_revenue</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q01_pricing_summary_report_nt">
+        <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q20_potential_part_promotion">
+        <output-dir compare="Text">q20_potential_part_promotion</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q21_suppliers_who_kept_orders_waiting">
+        <output-dir compare="Text">q21_suppliers_who_kept_orders_waiting</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q22_global_sales_opportunity">
+        <output-dir compare="Text">q22_global_sales_opportunity</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q02_minimum_cost_supplier">
+        <output-dir compare="Text">q02_minimum_cost_supplier</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q03_shipping_priority_nt">
+        <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q04_order_priority">
+        <output-dir compare="Text">q04_order_priority</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q05_local_supplier_volume">
+        <output-dir compare="Text">q05_local_supplier_volume</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q06_forecast_revenue_change">
+        <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q07_volume_shipping">
+        <output-dir compare="Text">q07_volume_shipping</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q08_national_market_share">
+        <output-dir compare="Text">q08_national_market_share</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q09_product_type_profit_nt">
+        <output-dir compare="Text">q09_product_type_profit_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="query-issue562">
+        <output-dir compare="Text">query-issue562</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="query-issue810">
+        <output-dir compare="Text">query-issue810</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="query-issue810-2">
+        <output-dir compare="Text">query-issue810-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="query-issue810-3">
+        <output-dir compare="Text">query-issue810-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="query-issue827">
+        <output-dir compare="Text">query-issue827</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="query-issue827-2">
+        <output-dir compare="Text">query-issue827-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch">
+      <compilation-unit name="q01-ASTERIXDB-830">
+        <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="tpch-sql">
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="distinct_by">
+        <output-dir compare="Text">distinct_by</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="group_no_agg">
+        <output-dir compare="Text">group_no_agg</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="nest_aggregate">
+        <output-dir compare="Text">nest_aggregate</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="nest_aggregate2">
+        <output-dir compare="Text">nest_aggregate2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="query-issue638">
+        <output-dir compare="Text">query-issue638</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="query-issue785">
+        <output-dir compare="Text">query-issue785</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="query-issue785-2">
+        <output-dir compare="Text">query-issue785-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="query-issue786">
+        <output-dir compare="Text">query-issue786</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="query-issue601">
+        <output-dir compare="Text">query-issue601</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q10_returned_item">
+        <output-dir compare="Text">q10_returned_item</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q10_returned_item_int64">
+        <output-dir compare="Text">q10_returned_item_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q11_important_stock">
+        <output-dir compare="Text">q11_important_stock</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q12_shipping">
+        <output-dir compare="Text">q12_shipping</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q13_customer_distribution">
+        <output-dir compare="Text">q13_customer_distribution</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q14_promotion_effect">
+        <output-dir compare="Text">q14_promotion_effect</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q15_top_supplier">
+        <output-dir compare="Text">q15_top_supplier</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q16_parts_supplier_relationship">
+        <output-dir compare="Text">q16_parts_supplier_relationship</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q17_small_quantity_order_revenue">
+        <output-dir compare="Text">q17_small_quantity_order_revenue</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q17_large_gby_variant">
+        <output-dir compare="Text">q17_large_gby_variant</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q18_large_volume_customer">
+        <output-dir compare="Text">q18_large_volume_customer</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q19_discounted_revenue">
+        <output-dir compare="Text">q19_discounted_revenue</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q01_pricing_summary_report_nt">
+        <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q20_potential_part_promotion">
+        <output-dir compare="Text">q20_potential_part_promotion</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q21_suppliers_who_kept_orders_waiting">
+        <output-dir compare="Text">q21_suppliers_who_kept_orders_waiting</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q22_global_sales_opportunity">
+        <output-dir compare="Text">q22_global_sales_opportunity</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q02_minimum_cost_supplier">
+        <output-dir compare="Text">q02_minimum_cost_supplier</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q03_shipping_priority_nt">
+        <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q04_order_priority">
+        <output-dir compare="Text">q04_order_priority</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q05_local_supplier_volume">
+        <output-dir compare="Text">q05_local_supplier_volume</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q06_forecast_revenue_change">
+        <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q07_volume_shipping">
+        <output-dir compare="Text">q07_volume_shipping</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q08_national_market_share">
+        <output-dir compare="Text">q08_national_market_share</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="q09_product_type_profit_nt">
+        <output-dir compare="Text">q09_product_type_profit_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="query-issue562">
+        <output-dir compare="Text">query-issue562</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="query-issue810">
+        <output-dir compare="Text">query-issue810</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="query-issue810-2">
+        <output-dir compare="Text">query-issue810-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql">
+      <compilation-unit name="query-issue810-3">
+        <output-dir compare="Text">query-issue810-3</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="tpch-sql-sugar">
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="distinct_by">
+        <output-dir compare="Text">distinct_by</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="group_no_agg">
+        <output-dir compare="Text">group_no_agg</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="nest_aggregate">
+        <output-dir compare="Text">nest_aggregate</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="nest_aggregate2">
+        <output-dir compare="Text">nest_aggregate2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="query-issue638">
+        <output-dir compare="Text">query-issue638</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="query-issue785">
+        <output-dir compare="Text">query-issue785</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="query-issue785-2">
+        <output-dir compare="Text">query-issue785-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="query-issue786">
+        <output-dir compare="Text">query-issue786</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="query-issue601">
+        <output-dir compare="Text">query-issue601</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q10_returned_item">
+        <output-dir compare="Text">q10_returned_item</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q10_returned_item_int64">
+        <output-dir compare="Text">q10_returned_item_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q11_important_stock">
+        <output-dir compare="Text">q11_important_stock</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q12_shipping">
+        <output-dir compare="Text">q12_shipping</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q12_shipping_broadcast">
+        <output-dir compare="Text">q12_shipping</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q13_customer_distribution">
+        <output-dir compare="Text">q13_customer_distribution</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q14_promotion_effect">
+        <output-dir compare="Text">q14_promotion_effect</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q15_top_supplier">
+        <output-dir compare="Text">q15_top_supplier</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q16_parts_supplier_relationship">
+        <output-dir compare="Text">q16_parts_supplier_relationship</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q17_small_quantity_order_revenue">
+        <output-dir compare="Text">q17_small_quantity_order_revenue</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q17_large_gby_variant">
+        <output-dir compare="Text">q17_large_gby_variant</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q17_large_gby_variant_parameter">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>Invalid query parameter compiler.groupmemory -- value has to be greater than or equal to</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q18_large_volume_customer">
+        <output-dir compare="Text">q18_large_volume_customer</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q19_discounted_revenue">
+        <output-dir compare="Text">q19_discounted_revenue</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q01_pricing_summary_report_nt">
+        <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q01_pricing_summary_report_2">
+        <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q01_pricing_summary_report_parallelism">
+        <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q01_pricing_summary_report_parameter">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>Invalid query parameter compiler.sortmemory -- value has to be greater than or equal to</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q20_potential_part_promotion">
+        <output-dir compare="Text">q20_potential_part_promotion</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q21_suppliers_who_kept_orders_waiting">
+        <output-dir compare="Text">q21_suppliers_who_kept_orders_waiting</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q22_global_sales_opportunity">
+        <output-dir compare="Text">q22_global_sales_opportunity</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q02_minimum_cost_supplier">
+        <output-dir compare="Text">q02_minimum_cost_supplier</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q03_shipping_priority_nt">
+        <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q04_order_priority">
+        <output-dir compare="Text">q04_order_priority</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q05_local_supplier_volume">
+        <output-dir compare="Text">q05_local_supplier_volume</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q06_forecast_revenue_change">
+        <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q07_volume_shipping">
+        <output-dir compare="Text">q07_volume_shipping</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q08_national_market_share">
+        <output-dir compare="Text">q08_national_market_share</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q09_product_type_profit_nt">
+        <output-dir compare="Text">q09_product_type_profit_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q09_product_type_profit_parameter">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>Invalid query parameter compiler.joinmemory -- value has to be greater than or equal to</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="query-issue562">
+        <output-dir compare="Text">query-issue562</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="query-issue810">
+        <output-dir compare="Text">query-issue810</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="query-issue810-2">
+        <output-dir compare="Text">query-issue810-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="query-issue810-3">
+        <output-dir compare="Text">query-issue810-3</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="tpch-with-index">
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="distinct_by">
+        <output-dir compare="Text">distinct_by</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="nest_aggregate">
+        <output-dir compare="Text">nest_aggregate</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="nest_aggregate2">
+        <output-dir compare="Text">nest_aggregate2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue785">
+        <output-dir compare="Text">query-issue785</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue785-2">
+        <output-dir compare="Text">query-issue785-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue786">
+        <output-dir compare="Text">query-issue786</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue601">
+        <output-dir compare="Text">query-issue601</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q10_returned_item">
+        <output-dir compare="Text">q10_returned_item</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q10_returned_item_int64">
+        <output-dir compare="Text">q10_returned_item_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q11_important_stock">
+        <output-dir compare="Text">q11_important_stock</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q12_shipping">
+        <output-dir compare="Text">q12_shipping</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q13_customer_distribution">
+        <output-dir compare="Text">q13_customer_distribution</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q14_promotion_effect">
+        <output-dir compare="Text">q14_promotion_effect</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q15_top_supplier">
+        <output-dir compare="Text">q15_top_supplier</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q16_parts_supplier_relationship">
+        <output-dir compare="Text">q16_parts_supplier_relationship</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q17_small_quantity_order_revenue">
+        <output-dir compare="Text">q17_small_quantity_order_revenue</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q17_large_gby_variant">
+        <output-dir compare="Text">q17_large_gby_variant</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q18_large_volume_customer">
+        <output-dir compare="Text">q18_large_volume_customer</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q19_discounted_revenue">
+        <output-dir compare="Text">q19_discounted_revenue</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q01_pricing_summary_report_nt">
+        <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q20_potential_part_promotion">
+        <output-dir compare="Text">q20_potential_part_promotion</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q21_suppliers_who_kept_orders_waiting">
+        <output-dir compare="Text">q21_suppliers_who_kept_orders_waiting</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q22_global_sales_opportunity">
+        <output-dir compare="Text">q22_global_sales_opportunity</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q02_minimum_cost_supplier">
+        <output-dir compare="Text">q02_minimum_cost_supplier</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q03_shipping_priority_nt">
+        <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q04_order_priority">
+        <output-dir compare="Text">q04_order_priority</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q04_order_priority_with_nodegroup">
+        <output-dir compare="Text">q04_order_priority</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q05_local_supplier_volume">
+        <output-dir compare="Text">q05_local_supplier_volume</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q06_forecast_revenue_change">
+        <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q07_volume_shipping">
+        <output-dir compare="Text">q07_volume_shipping</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q08_national_market_share">
+        <output-dir compare="Text">q08_national_market_share</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="q09_product_type_profit_nt">
+        <output-dir compare="Text">q09_product_type_profit_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue562">
+        <output-dir compare="Text">query-issue562</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue810">
+        <output-dir compare="Text">query-issue810</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue810-2">
+        <output-dir compare="Text">query-issue810-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue810-3">
+        <output-dir compare="Text">query-issue810-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue827">
+        <output-dir compare="Text">query-issue827</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-with-index">
+      <compilation-unit name="query-issue827-2">
+        <output-dir compare="Text">query-issue827-2</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="tpch-sql-like">
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="query-issue638">
+        <output-dir compare="Text">query-issue638</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="query-issue785">
+        <output-dir compare="Text">query-issue785</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="query-issue785-2">
+        <output-dir compare="Text">query-issue785-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="query-issue786">
+        <output-dir compare="Text">query-issue786</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="query-issue601">
+        <output-dir compare="Text">query-issue601</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q10_returned_item">
+        <output-dir compare="Text">q10_returned_item</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q10_returned_item_int64">
+        <output-dir compare="Text">q10_returned_item_int64</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q11_important_stock">
+        <output-dir compare="Text">q11_important_stock</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q12_shipping">
+        <output-dir compare="Text">q12_shipping</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q13_customer_distribution">
+        <output-dir compare="Text">q13_customer_distribution</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q14_promotion_effect">
+        <output-dir compare="Text">q14_promotion_effect</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q15_top_supplier">
+        <output-dir compare="Text">q15_top_supplier</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q16_parts_supplier_relationship">
+        <output-dir compare="Text">q16_parts_supplier_relationship</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q17_small_quantity_order_revenue">
+        <output-dir compare="Text">q17_small_quantity_order_revenue</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q17_large_gby_variant">
+        <output-dir compare="Text">q17_large_gby_variant</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q18_large_volume_customer">
+        <output-dir compare="Text">q18_large_volume_customer</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q19_discounted_revenue">
+        <output-dir compare="Text">q19_discounted_revenue</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q01_pricing_summary_report_nt">
+        <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q20_potential_part_promotion">
+        <output-dir compare="Text">q20_potential_part_promotion</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q21_suppliers_who_kept_orders_waiting">
+        <output-dir compare="Text">q21_suppliers_who_kept_orders_waiting</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q22_global_sales_opportunity">
+        <output-dir compare="Text">q22_global_sales_opportunity</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q02_minimum_cost_supplier">
+        <output-dir compare="Text">q02_minimum_cost_supplier</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q03_shipping_priority_nt">
+        <output-dir compare="Text">q03_shipping_priority_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q04_order_priority">
+        <output-dir compare="Text">q04_order_priority</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q05_local_supplier_volume">
+        <output-dir compare="Text">q05_local_supplier_volume</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q06_forecast_revenue_change">
+        <output-dir compare="Text">q06_forecast_revenue_change</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q07_volume_shipping">
+        <output-dir compare="Text">q07_volume_shipping</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q08_national_market_share">
+        <output-dir compare="Text">q08_national_market_share</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-like">
+      <compilation-unit name="q09_product_type_profit_nt">
+        <output-dir compare="Text">q09_product_type_profit_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="writers">
+    <test-case FilePath="writers">
+      <compilation-unit name="print_01">
+        <output-dir compare="Text">print_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--  TODO(madhusudancs): Enable this test when REST API supports serialized output support.
+            <test-case FilePath="writers">
+              <compilation-unit name="serialized_01">
+                <output-dir compare="Text">serialized_01</output-dir>
+              </compilation-unit>
+            </test-case>
+        -->
+  </test-group>
+  <test-group name="cross-dataverse">
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="cross-dv01">
+        <output-dir compare="Text">cross-dv01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="cross-dv02">
+        <output-dir compare="Text">cross-dv02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="cross-dv03">
+        <output-dir compare="Text">cross-dv03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="cross-dv04">
+        <output-dir compare="Text">cross-dv04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="cross-dv07">
+        <output-dir compare="Text">cross-dv07</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--NotImplementedException: No binary comparator factory implemented for type OBJECT.
+        <test-case FilePath="cross-dataverse">
+          <compilation-unit name="cross-dv08">
+            <output-dir compare="Text">cross-dv08</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="cross-dv09">
+        <output-dir compare="Text">cross-dv09</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="cross-dv11">
+        <output-dir compare="Text">cross-dv11</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="cross-dv12">
+        <output-dir compare="Text">cross-dv12</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="cross-dv14">
+        <output-dir compare="Text">cross-dv14</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="cross-dv15">
+        <output-dir compare="Text">cross-dv15</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--NotImplementedException: No binary comparator factory implemented for type OBJECT.
+        <test-case FilePath="cross-dataverse">
+          <compilation-unit name="cross-dv17">
+            <output-dir compare="Text">cross-dv17</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <!--NotImplementedException: No binary comparator factory implemented for type OBJECT.
+        <test-case FilePath="cross-dataverse">
+          <compilation-unit name="cross-dv18">
+            <output-dir compare="Text">cross-dv18</output-dir>
+          </compilation-unit>
+        </test-case>
+        -->
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="cross-dv19">
+        <output-dir compare="Text">cross-dv19</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="cross-dv20">
+        <output-dir compare="Text">cross-dv20</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="insert_across_dataverses">
+        <output-dir compare="Text">insert_across_dataverses</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="join_across_dataverses">
+        <output-dir compare="Text">join_across_dataverses</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="drop-dataverse">
+        <output-dir compare="Text">drop-dataverse</output-dir>
+        <expected-error>ASX1147: Cannot drop dataverse: type a.a being used by dataset b.b1</expected-error>
+        <expected-error>ASX1147: Cannot drop dataverse: synonym a.s1 being used by function b.f1()</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="drop-type-used-elsewhere">
+        <output-dir compare="Text">drop-type-used-elsewhere</output-dir>
+        <expected-error>Cannot drop type a.a being used by dataset b.b1</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="drop-type-used-here-dataset">
+        <output-dir compare="Text">drop-type-used-here-dataset</output-dir>
+        <expected-error>Cannot drop type c.a being used by dataset c.a1</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="drop-type-used-here-type">
+        <output-dir compare="Text">drop-type-used-here-type</output-dir>
+        <expected-error>Cannot drop type c.a being used by type c.b</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="cross-dataverse">
+      <compilation-unit name="query-dataset-with-foreign-type">
+        <output-dir compare="Text">query-dataset-with-foreign-type</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="user-defined-functions">
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="bad-function-ddl-1">
+        <output-dir compare="Text">bad-function-ddl-1</output-dir>
+        <expected-error>Cannot find dataset TweetMessages in dataverse experiments nor an alias with name TweetMessages</expected-error>
+        <expected-error>Cannot find dataset TweetMessages2 in dataverse experiments2 nor an alias with name TweetMessages2</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="bad-function-ddl-2">
+        <output-dir compare="Text">bad-function-ddl-2</output-dir>
+        <expected-error>Cannot find dataset TweetMessages in dataverse experiments2 nor an alias with name TweetMessages</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="bad-function-ddl-3">
+        <output-dir compare="Text">bad-function-ddl-3</output-dir>
+        <expected-error>Cannot find dataset TweetMessages in dataverse experiments nor an alias with name TweetMessages</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="bad-function-ddl-4">
+        <output-dir compare="Text">bad-function-ddl-4</output-dir>
+        <expected-error>Cannot find dataset TweetMessages in dataverse experients nor an alias with name TweetMessages</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="bad-function-ddl-5">
+        <output-dir compare="Text">bad-function-ddl-5</output-dir>
+        <expected-error>ASX1081: Cannot find function with signature experiments.function_that_does_not_exist()</expected-error>
+        <expected-error>ASX1081: Cannot find function with signature experiments2.function_that_does_not_exist()</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="bad-function-ddl-6">
+        <output-dir compare="Text">bad-function-ddl-6</output-dir>
+        <expected-error>ASX1081: Cannot find function with signature experiments.function_that_does_not_exist(2)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="bad-function-ddl-7">
+        <output-dir compare="Text">bad-function-ddl-7</output-dir>
+        <expected-error>Cannot find dataset TweetMessaes in dataverse experiments nor an alias with name TweetMessaes</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="bad-function-ddl-8">
+        <output-dir compare="Text">bad-function-ddl-8</output-dir>
+        <expected-error>Cannot find dataset TweetMessaes in dataverse experiments nor an alias with name TweetMessaes</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="bad-function-ddl-9">
+        <output-dir compare="Text">bad-function-ddl-9</output-dir>
+        <expected-error>ASX1081: Cannot find function with signature experiments.function_that_does_not_exist()</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="bad-function-ddl-10">
+        <output-dir compare="Text">bad-function-ddl-10</output-dir>
+        <expected-error>ASX1081: Cannot find function with signature experiments.f0(2)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="bad-function-ddl-11">
+        <output-dir compare="Text">bad-function-ddl-11</output-dir>
+        <expected-error>ASX1001: Syntax error: Unexpected type declaration for parameter a in function myfn001</expected-error>
+        <expected-error>ASX1001: Syntax error: Unexpected return type declaration for function myfn002</expected-error>
+        <expected-error>ASX1001: Syntax error: Unexpected return type declaration for function myfn003</expected-error>
+        <expected-error>ASX1001: Syntax error: Unexpected IF NOT EXISTS (in line 29, at column 45)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="check-dependencies-1">
+        <output-dir compare="Text">check-dependencies-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="create-or-replace-function-1">
+        <output-dir compare="Text">create-or-replace-function-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="drop-dependency-1">
+        <output-dir compare="Text">drop-dependency-1</output-dir>
+        <expected-error>ASX1147: Cannot drop dataverse: function C.f1(2) being used by function B.f0(2)</expected-error>
+        <expected-error>ASX1147: Cannot drop dataverse: function C.f2(...) being used by function B.f3(2)</expected-error>
+        <expected-error>ASX1147: Cannot drop dataverse: function C.f4(2) being used by function B.f5(...)</expected-error>
+        <expected-error>ASX1147: Cannot drop dataverse: function C.f6(...) being used by function B.f7(...)</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>ASX1147: Cannot drop dataverse: dataset (or view) C.TweetMessages being used by function B.f2(2)</expected-error>
+        <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) C.TweetMessages being used by function B.f3(...)</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>
+        <expected-error>Cannot drop function C.f3(...) being used by function B.f2(2)</expected-error>
+        <expected-error>Cannot drop function C.f5(2) being used by function B.f4(...)</expected-error>
+        <expected-error>Cannot drop function C.f7(...) being used by function B.f6(...)</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>ASX1148: Cannot drop dataset C.TweetMessages being used by function B.f2(2)</expected-error>
+        <expected-error>ASX1148: Cannot drop dataset C.TweetMessages being used by function B.f2(...)</expected-error>
+        <expected-error>ASX1148: Cannot drop synonym C.TweetMessagesSyn being used by function B.f2(2)</expected-error>
+        <expected-error>ASX1148: Cannot drop synonym C.TweetMessagesSyn being used by function B.f2(...)</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>
+        <expected-error>Cannot drop function C.f1(2) being used by function C.f0(...)</expected-error>
+        <expected-error>Cannot drop function C.f1(...) being used by function C.f0(2)</expected-error>
+        <expected-error>Cannot drop function C.f1(...) being used by function C.f0(...)</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>ASX1148: Cannot drop dataset C.TweetMessages being used by function C.f2(2)</expected-error>
+        <expected-error>ASX1148: Cannot drop dataset C.TweetMessages being used by function C.f2(...)</expected-error>
+        <expected-error>ASX1148: Cannot drop synonym C.TweetMessagesSyn being used by function C.f2(2)</expected-error>
+        <expected-error>ASX1148: Cannot drop synonym C.TweetMessagesSyn being used by function C.f2(...)</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="drop-function-1">
+        <output-dir compare="Text">drop-function-1</output-dir>
+        <expected-error>ASX1081: Cannot find function with signature experiments.my_sum(2)</expected-error>
+        <expected-error>ASX1081: Cannot find function with signature experiments.my_sum_va(2)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="single-line-definition">
+        <output-dir compare="Text">single-line-definition</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="query-ASTERIXDB-1298">
+        <output-dir compare="Text">query-ASTERIXDB-1298</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="query-ASTERIXDB-1652">
+        <output-dir compare="Text">query-ASTERIXDB-1652-2</output-dir>
+        <expected-error>ASX1063: Cannot find dataverse with name test</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="query-ASTERIXDB-1652-2">
+        <output-dir compare="Text">query-ASTERIXDB-1652-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="query-issue218-2">
+        <output-dir compare="Text">query-issue218-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="query-issue218">
+        <output-dir compare="Text">query-issue218</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="query-issue201">
+        <output-dir compare="Text">query-issue201</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="query-issue172">
+        <output-dir compare="Text">query-issue172</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="query-issue455">
+        <output-dir compare="Text">query-issue455</output-dir>
+        <expected-error>ASX1081: Cannot find function with signature test.printName()</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="query-issue489">
+        <output-dir compare="Text">query-issue489</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf01">
+        <output-dir compare="Text">udf01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf02">
+        <output-dir compare="Text">udf02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf03">
+        <output-dir compare="Text">udf03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf04">
+        <output-dir compare="Text">udf04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf05">
+        <output-dir compare="Text">udf05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf06">
+        <output-dir compare="Text">udf06</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf07">
+        <output-dir compare="Text">udf07</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf08">
+        <output-dir compare="Text">udf08</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf09">
+        <output-dir compare="Text">udf09</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf10">
+        <output-dir compare="Text">udf10</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf11">
+        <output-dir compare="Text">udf11</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf12">
+        <output-dir compare="Text">udf12</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf13">
+        <output-dir compare="Text">udf13</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf14">
+        <output-dir compare="Text">udf14</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf15">
+        <output-dir compare="Text">udf15</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf16">
+        <output-dir compare="Text">udf16</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf17">
+        <output-dir compare="Text">udf17</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf18">
+        <output-dir compare="Text">udf18</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf19">
+        <output-dir compare="Text">udf19</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf20">
+        <output-dir compare="Text">udf20</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf21">
+        <output-dir compare="Text">udf21</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf22">
+        <output-dir compare="Text">udf22</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf23">
+        <output-dir compare="Text">udf23</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf24">
+        <output-dir compare="Text">udf24</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf25">
+        <output-dir compare="Text">udf25</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf27">
+        <output-dir compare="Text">udf27</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf28">
+        <output-dir compare="Text">udf28</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf29">
+        <output-dir compare="Text">udf29</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!-- Need to verify the expected exception -->
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf30">
+        <output-dir compare="Text">udf30</output-dir>
+        <expected-error>ASX1073: Cannot resolve alias reference for undefined identifier y (in line 30, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf31">
+        <output-dir compare="Text">udf31</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf32_metadata">
+        <output-dir compare="Text">udf32_metadata</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf33_overloading">
+        <output-dir compare="Text">udf33_overloading</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf34">
+        <output-dir compare="Text">udf34</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf35_varargs_misc">
+        <output-dir compare="Text">udf35_varargs_misc</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf36_in_with">
+        <output-dir compare="Text">udf36_in_with</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf37_recursion">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1149: Illegal function or view recursion (in line 24, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function or view recursion (in line 25, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function or view recursion (in line 26, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function or view recursion (in line 27, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function or view recursion (in line 28, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function or view recursion (in line 29, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function or view recursion (in line 30, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function or view recursion (in line 31, at column 1)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf38_no_recursion">
+        <output-dir compare="Text">udf38_no_recursion</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf39_illegal_call">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1150: Illegal use of function test.f1a(1) (in line 32, at column 26)</expected-error>
+        <expected-error>ASX1150: Illegal use of function test.f2a(...) (in line 29, at column 28)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="f01">
+        <output-dir compare="Text">f01</output-dir>
+        <expected-error>ASX1081: Cannot find function with signature test.tinyint()</expected-error>
+      </compilation-unit>
+    </test-case>
+    <!-- This test case is not valid anymore since we do not required "IMPORT_PRIVATE_FUNCTIONS" flag anymore -->
+    <!-- <test-case FilePath="user-defined-functions">
+            <compilation-unit name="invoke-private-function">
+                <output-dir compare="Text">invoke-private-function</output-dir>
+            </compilation-unit>
+        </test-case>-->
+    <!--
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="query-ASTERIXDB-1308-1">
+        <output-dir compare="Text">query-ASTERIXDB-1308-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    -->
+    <!-- <test-case FilePath="user-defined-functions">
+          <compilation-unit name="query-ASTERIXDB-1308-2">
+              <output-dir compare="Text">query-ASTERIXDB-1308-2</output-dir>
+          </compilation-unit>
+      </test-case>  -->
+    <!-- This test case is not valid anymore since we do not required "IMPORT_PRIVATE_FUNCTIONS" flag anymore -->
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="query-ASTERIXDB-1317">
+        <output-dir compare="Text">query-ASTERIXDB-1317</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="view">
+    <test-case FilePath="view">
+      <compilation-unit name="create-view-1">
+        <output-dir compare="Text">create-view-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="view">
+      <compilation-unit name="create-view-2-negative">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1063: Cannot find dataverse with name test (in line 24, at column 1)</expected-error>
+        <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>create view test.v1 as select * from range(1,2) r order by;<< Encountered ";" at column 59]]></expected-error>
+        <expected-error>ASX1081: Cannot find function with signature test.undefined_range(2) (in line 25, at column 38)</expected-error>
+        <expected-error>ASX1160: A view with this name test.v1 already exists (in line 26, at column 1)</expected-error>
+        <expected-error>ASX1072: A dataset with name ds1 already exists in dataverse test (in line 30, at column 1)</expected-error>
+        <expected-error>ASX1072: A dataset with name ds2 already exists in dataverse test (in line 30, at column 1)</expected-error>
+        <expected-error>ASX1160: A view with this name test.ds1 already exists (in line 27, at column 1)</expected-error>
+        <expected-error><![CDATA[ASX1001: Syntax error: Unexpected IF NOT EXISTS (in line 25, at column 1)]]></expected-error>
+        <expected-error>ASX1149: Illegal function or view recursion (in line 31, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function or view recursion (in line 32, at column 1)</expected-error>
+        <expected-error>ASX1149: Illegal function or view recursion (in line 33, at column 1)</expected-error>
+        <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>create view test.v1 primary key (r) not enforced as<< Encountered "primary" at column 21]]></expected-error>
+        <expected-error><![CDATA[ASX1001: Syntax error: In line 28 >>create view test.v2 foreign key (r) references v1 not enforced as<< Encountered <IDENTIFIER> "foreign" at column 21]]></expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="view">
+      <compilation-unit name="create-view-3-typed">
+        <output-dir compare="Text">create-view-3-typed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="view" check-warnings="true">
+      <compilation-unit name="create-view-4-typed-warn">
+        <output-dir compare="Text">create-view-4-typed-warn</output-dir>
+        <expected-warn>ASX0006: Invalid format for tinyint in a (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for smallint in b (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for integer in c (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for bigint in d (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for float in e (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for double in f (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for datetime in g (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for date in h (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for time in j (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for duration in k (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for yearmonthduration in m (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for daytimeduration in n (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: boolean-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int8-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int16-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int32-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: int64-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: float-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: double-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: datetime-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: date-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: time-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: duration-default-null() cannot process input type date (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: year-month-duration-default-null() cannot process input type date (in line 30, at column 6)</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: day-time-duration-default-null() cannot process input type date (in line 30, at column 6)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="view" check-warnings="true">
+      <compilation-unit name="create-view-5-typed-warn">
+        <output-dir compare="Text">create-view-5-typed-warn</output-dir>
+        <expected-warn>ASX0006: Invalid format for datetime in a (in line 27, at column 6)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for date in b (in line 27, at column 6)</expected-warn>
+        <expected-warn>ASX0006: Invalid format for time in c (in line 27, at column 6)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="view">
+      <compilation-unit name="create-view-6-typed-negative">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1082: Cannot find datatype with name unknown_dv.t1</expected-error>
+        <expected-error>ASX1082: Cannot find datatype with name test.t1_unknown</expected-error>
+        <expected-error>ASX1079: Compilation error: view type cannot have open fields (in line 29, at column 1)</expected-error>
+        <expected-error>ASX1004: Unsupported type: view cannot process input type t1_a (in line 30, at column 1)</expected-error>
+        <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>create view test.v1(r bigint, a [bigint]) default null as<< Encountered "[" at column 33]]></expected-error>
+        <expected-error>ASX1092: Parameter date_illegal_property_name cannot be set (in line 25, at column 1)</expected-error>
+        <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>create view test.v1(r bigint) as<< Encountered "as" at column 31]]></expected-error>
+        <expected-error><![CDATA[ASX1014: Field 'unknown_field' is not found (in line 25, at column 1)]]></expected-error>
+        <expected-error><![CDATA[ASX1014: Field 'unknown_field_2' is not found (in line 25, at column 1)]]></expected-error>
+        <expected-error><![CDATA[ASX1001: Syntax error: In line 28 >>  as select r from range(1,2) r;<< Encountered "as" at column 3]]></expected-error>
+        <expected-error><![CDATA[ASX0013: Duplicate field name 'r' (in line 25, at column 20)]]></expected-error>
+        <expected-error><![CDATA[ASX1167: Cannot change primary key of view test1.employee_v1 (in line 38, at column 1)]]></expected-error>
+        <expected-error><![CDATA[ASX1162: Invalid primary key definition (in line 25, at column 1)]]></expected-error>
+        <expected-error><![CDATA[ASX1162: Invalid primary key definition (in line 26, at column 1)]]></expected-error>
+        <expected-error><![CDATA[ASX1001: Syntax error: In line 36 >>  as employee;<< Encountered "as" at column 3]]></expected-error>
+        <expected-error><![CDATA[ASX1165: Invalid foreign key definition: view test1.employee_v1 does not have a primary key (in line 32, at column 1)]]></expected-error>
+        <expected-error><![CDATA[ASX1166: Invalid foreign key definition: foreign key does not match primary key of view test1.employee_v1 (in line 34, at column 1)]]></expected-error>
+        <expected-error><![CDATA[ASX1166: Invalid foreign key definition: foreign key does not match primary key of view test1.employee_v2 (in line 34, at column 1)]]></expected-error>
+        <expected-error><![CDATA[ASX1164: Invalid foreign key definition (in line 34, at column 1)]]></expected-error>
+        <expected-error><![CDATA[ASX1063: Cannot find dataverse with name test3 (in line 42, at column 1)]]></expected-error>
+        <expected-error><![CDATA[ASX1159: Cannot find view with name test1.employee_v3 (in line 42, at column 1)]]></expected-error>
+        <expected-error><![CDATA[ASX1159: Cannot find view with name test1.employee (in line 43, at column 1)]]></expected-error>
+        <expected-error><![CDATA[ASX1164: Invalid foreign key definition (in line 43, at column 1)]]></expected-error>
+        <expected-error><![CDATA[ASX1164: Invalid foreign key definition (in line 39, at column 1)]]></expected-error>
+        <expected-error><![CDATA[ASX1166: Invalid foreign key definition: foreign key does not match primary key of view test1.employee_v1 (in line 39, at column 1)]]></expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="view">
+      <compilation-unit name="create-view-7-foreign-key">
+        <output-dir compare="Text">create-view-7-foreign-key</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="view">
+      <compilation-unit name="drop-dataverse-1">
+        <output-dir compare="Text">drop-dataverse-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="view">
+      <compilation-unit name="drop-dataverse-2-negative">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) test2.v2 being used by view test1.v1</expected-error>
+        <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) test2.v2 being used by function test1.f1()</expected-error>
+        <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) test2.ds2 being used by view test1.v1</expected-error>
+        <expected-error>ASX1147: Cannot drop dataverse: function test2.f2() being used by view test1.v1</expected-error>
+        <expected-error>ASX1147: Cannot drop dataverse: synonym test2.s3 being used by view test1.v1</expected-error>
+        <expected-error>ASX1147: Cannot drop dataverse: type test2.t1 being used by dataset test1.v1</expected-error>
+        <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) test2.employee_v2 being used by view test1.employee_v1</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="view">
+      <compilation-unit name="drop-view-1">
+        <output-dir compare="Text">drop-view-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="view">
+      <compilation-unit name="drop-view-2-negative">
+        <output-dir compare="Text">drop-view-2-negative</output-dir>
+        <expected-error>ASX1063: Cannot find dataverse with name test (in line 24, at column 1)</expected-error>
+        <expected-error>ASX1159: Cannot find view with name test.v1 (in line 25, at column 1)</expected-error>
+        <expected-error>ASX1050: Cannot find dataset with name v1 in dataverse test (in line 27, at column 1)</expected-error>
+        <expected-error>ASX1159: Cannot find view with name test.ds1 (in line 30, at column 1)</expected-error>
+        <expected-error>ASX1148: Cannot drop dataset test2.ds2 being used by view test1.v1</expected-error>
+        <expected-error>ASX1148: Cannot drop function test2.f2() being used by view test1.v1</expected-error>
+        <expected-error>ASX1148: Cannot drop synonym test2.s2 being used by view test1.v1</expected-error>
+        <expected-error>ASX1148: Cannot drop view test2.v2 being used by view test1.v1</expected-error>
+        <expected-error>ASX1148: Cannot drop view test2.v2 being used by function test1.f1()</expected-error>
+        <expected-error>ASX1148: Cannot drop type test2.t1 being used by dataset test1.v1</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="view">
+      <compilation-unit name="view-1">
+        <output-dir compare="Text">view-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="view">
+      <compilation-unit name="view-2-negative">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1050: Cannot find dataset with name v1 in dataverse test1 (in line 24, at column 17)</expected-error>
+        <expected-error>ASX1050: Cannot find dataset with name v2 in dataverse test1 (in line 24, at column 17)</expected-error>
+        <expected-error>ASX1050: Cannot find dataset with name v3 in dataverse test1 (in line 24, at column 1)</expected-error>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="load">
+    <test-case FilePath="load">
+      <compilation-unit name="load_non-empty_index">
+        <output-dir compare="Text">load_non-empty_index</output-dir>
+        <expected-error>HYR0034: Cannot load an index that is not empty</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="csv_01">
+        <output-dir compare="Text">csv_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="csv_02">
+        <output-dir compare="Text">csv_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="csv_03">
+        <output-dir compare="Text">csv_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="csv_04">
+        <output-dir compare="Text">csv_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="csv_05">
+        <output-dir compare="Text">csv_05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="csv_06">
+        <output-dir compare="Text">csv_06</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="csv_07">
+        <output-dir compare="Text">csv_07</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="csv_08_header_cr">
+        <output-dir compare="Text">csv_08</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="csv_08_header_lf">
+        <output-dir compare="Text">csv_08</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="csv_08_header_crlf">
+        <output-dir compare="Text">csv_08</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <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">
+      <compilation-unit name="issue289_query">
+        <output-dir compare="Text">issue289_query</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="issue650_query">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>Cannot find dataset with name Users in dataverse fuzzyjoin</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="type_promotion_0">
+        <output-dir compare="Text">type_promotion_0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="escapes01">
+        <output-dir compare="Text">escapes01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="escapes02">
+        <output-dir compare="Text">escapes02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="escapes-err-1"><!-- Exception is never thrown!!!. needs to be investigated -->
+        <output-dir compare="Text">none</output-dir>
+        <!-- <expected-error>org.apache.hyracks.api.exceptions.HyracksException</expected-error> -->
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="query-issue244">
+        <output-dir compare="Text">query-issue244</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <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">
+      <compilation-unit name="issue610_adm_token_end_collection">
+        <output-dir compare="Text">issue610_adm_token_end_collection</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="adm_binary">
+        <output-dir compare="Text">adm_binary</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="dataset-with-meta">
+        <output-dir compare="Text">dataset-with-meta</output-dir>
+        <expected-error>ASX1079: Compilation error: DatasetWithMeta: load dataset is not supported on datasets with meta records (in line 27, at column 1)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="escapes-err-1">
+        <output-dir compare="Text">escapes-err-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="escapes01">
+        <output-dir compare="Text">escapes01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="escapes02">
+        <output-dir compare="Text">escapes02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="file-not-found">
+        <output-dir compare="Text">file-not-found</output-dir>
+        <expected-error>ASX3077: bla: path not found</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="type_promotion_0">
+        <output-dir compare="Text">type_promotion_0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="utf8">
+        <output-dir compare="Text">utf8</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="hints">
+    <test-case FilePath="hints">
+      <compilation-unit name="issue_251_dataset_hint_5">
+        <output-dir compare="Text">issue_251_dataset_hint_5</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="hints">
+      <compilation-unit name="issue_251_dataset_hint_7">
+        <output-dir compare="Text">issue_251_dataset_hint_7</output-dir>
+      </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-case FilePath="function">
+      <compilation-unit name="drop_if_exists">
+        <output-dir compare="Text">drop_if_exists</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="feeds">
+    <test-case FilePath="feeds">
+      <compilation-unit name="feeds_07">
+        <output-dir compare="Text">feeds_07</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="feeds_08">
+        <output-dir compare="Text">feeds_08</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="feeds_09">
+        <output-dir compare="Text">feeds_09</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="create-policy-from-file">
+        <output-dir compare="Text">create-policy-from-file</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="feeds_01">
+        <output-dir compare="Text">feeds_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="feeds_02">
+        <output-dir compare="Text">feeds_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="feeds_03">
+        <output-dir compare="Text">feeds_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="feeds_10">
+        <output-dir compare="Text">feeds_10</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="feeds_11">
+        <output-dir compare="Text">feeds_11</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="feeds_12">
+        <output-dir compare="Text">feeds_12</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="feeds_13">
+        <output-dir compare="Text">feeds_13</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="issue_230_feeds">
+        <output-dir compare="Text">issue_230_feeds</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="insert-feed">
+        <output-dir compare="Text">insert-feed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="insert-feed-with-pk-index">
+        <output-dir compare="Text">insert-feed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="connect-feed-with-function">
+        <output-dir compare="Text">connect-feed-with-function</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="change-feed-filter-on-meta-dataset">
+        <output-dir compare="Text">change-feed-filter-on-meta-dataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="change-feed-with-meta-pk-index">
+        <output-dir compare="Text">change-feed-with-meta-pk-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="change-feed-with-meta-with-mixed-index">
+        <output-dir compare="Text">change-feed-with-meta-with-mixed-index</output-dir>
+        <expected-error>Compilation error: Cannot create index on meta fields (in line 61, at column 35)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="change-feed-with-meta-pk-in-meta-index-with-missing-after-ingest">
+        <output-dir compare="Text">change-feed-with-meta-pk-in-meta-index-with-missing-after-ingest</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="change-feed-with-meta-pk-in-meta-open-index-with-missing">
+        <output-dir compare="Text">change-feed-with-meta-pk-in-meta-open-index-with-missing</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="change-feed-with-meta-open-index-in-meta">
+        <output-dir compare="Text">change-feed-with-meta-open-index-in-meta</output-dir>
+        <expected-error>Compilation error: Cannot create index on meta fields (in line 60, at column 34)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="change-feed-with-meta-pk-in-meta-open-index-in-value">
+        <output-dir compare="Text">change-feed-with-meta-pk-in-meta-open-index-in-value</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="change-feed-with-meta-pk-in-meta-index-after-ingest">
+        <output-dir compare="Text">change-feed-with-meta-pk-in-meta-index-after-ingest</output-dir>
+        <expected-error>Compilation error: Cannot create index on meta fields (in line 27, at column 37)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="change-feed-with-meta-pk-in-meta-index-in-meta">
+        <output-dir compare="Text">change-feed-with-meta-pk-in-meta-index-in-meta</output-dir>
+        <expected-error>Compilation error: Cannot create index on meta fields (in line 60, at column 37)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="change-feed-with-meta-csv">
+        <output-dir compare="Text">change-feed-with-meta-csv</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="change-feed-with-meta-pk-in-meta">
+        <output-dir compare="Text">change-feed-with-meta-pk-in-meta</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="change-feed">
+        <output-dir compare="Text">change-feed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="drop-nonexistent-feed">
+        <output-dir compare="Text">drop-nonexistent-feed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="twitter-feed">
+        <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">
+      <compilation-unit name="revised-tweet-parser">
+        <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">
+      <compilation-unit name="connect-feed">
+        <output-dir compare="Text">connect-feed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="change-feed-with-meta-pk-in-meta">
+        <output-dir compare="Text">change-feed-with-meta-pk-in-meta</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="feed-with-filtered-dataset">
+        <output-dir compare="Text">feed-with-filtered-dataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="change-feed">
+        <output-dir compare="Text">change-feed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="drop-dataverse-with-disconnected-feed">
+        <output-dir compare="Text">drop-dataverse-with-disconnected-feed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="feed-push-socket">
+        <output-dir compare="Text">feed-push-socket</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="insert-feed">
+        <output-dir compare="Text">insert-feed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="start-feed">
+        <output-dir compare="Text">start-feed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <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">
+      <compilation-unit name="push-socket-with-auuid">
+        <output-dir compare="Text">push-socket-with-auuid</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="disconnect-live-feed">
+        <output-dir compare="Text">disconnect-live-feed</output-dir>
+        <expected-error>This operation cannot be done when Feed</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="connect-live-feed">
+        <output-dir compare="Text">connect-live-feed</output-dir>
+        <expected-error>This operation cannot be done when Feed</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="record-reader-with-malformed-input-stream">
+        <output-dir compare="Text">record-reader-with-malformed-input-stream</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="feed-with-undefined-function">
+        <output-dir compare="Text">feed-with-undefined-function</output-dir>
+        <expected-error>Cannot find function</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="drop-function-used-by-feed">
+        <output-dir compare="Text">drop-function-used-by-feed</output-dir>
+        <expected-error>ASX1148: Cannot drop function experiments.test_func0(1) being used by feed connection experiments.UserFeed</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="drop-function-no-longer-used-by-feed">
+        <output-dir compare="Text">drop-function-used-by-feed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="drop-dataverse-with-function-used-by-feed">
+        <output-dir compare="Text">drop-dataverse-with-function-used-by-feed</output-dir>
+        <expected-error>ASX1147: Cannot drop dataverse: function fundv.test_func0(1) being used by feed connection feeddv.UserFeed</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="http_feed">
+        <output-dir compare="Text">http_feed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="http_feed_json">
+        <output-dir compare="Text">http_feed_json</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="change-feed-with-where-on-meta">
+        <output-dir compare="Text">change-feed-with-where-on-meta</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="meta">
+    <test-case FilePath="meta">
+      <compilation-unit name="meta_in_with_clause">
+        <output-dir compare="Text">meta_in_with_clause</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="meta">
+      <compilation-unit name="resolving_pk_with_meta">
+        <output-dir compare="Text">resolving_pk_with_meta</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="meta">
+      <compilation-unit name="meta_after_gby">
+        <output-dir compare="Text">meta_after_gby</output-dir>
+        <expected-error>Compilation error: Inappropriate use of function 'meta'. For example, after GROUP BY (in line 29, at column 21)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="meta">
+      <compilation-unit name="query_dataset_with_meta-1">
+        <output-dir compare="Text">query_dataset_with_meta-1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="meta">
+      <compilation-unit name="query_dataset_with_meta-2">
+        <output-dir compare="Text">query_dataset_with_meta-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="meta">
+      <compilation-unit name="query_dataset_with_meta_failure">
+        <output-dir compare="Text">query_dataset_with_meta_failure</output-dir>
+        <expected-error>ASX1079: Compilation error: Cannot resolve ambiguous meta function call. There are more than one dataset choice (in line 24, at column 7)</expected-error>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="big-object">
+    <test-case FilePath="big-object">
+      <compilation-unit name="big_object_sort">
+        <output-dir compare="Text">big_object_sort</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="big-object">
+      <compilation-unit name="big_object_groupby">
+        <output-dir compare="Text">big_object_groupby</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="big-object">
+      <compilation-unit name="big_object_groupby-2">
+        <output-dir compare="Text">big_object_groupby-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="big-object">
+      <compilation-unit name="big_object_join">
+        <output-dir compare="Text">big_object_join</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="big-object">
+      <compilation-unit name="big_object_join_low_memory_err">
+        <output-dir compare="Text">big_object_join</output-dir>
+        <expected-error>HYR0123: Insufficient memory is provided for the join operators, please increase the join memory budget.</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="big-object">
+      <compilation-unit name="big_object_load_20M">
+        <output-dir compare="Text">big_object_load_20M</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="big-object">
+      <compilation-unit name="big_object_bulkload">
+        <output-dir compare="Text">big_object_bulkload</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="big-object">
+      <compilation-unit name="big_object_feed_20M">
+        <output-dir compare="Text">big_object_feed_20M</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="big-object">
+      <compilation-unit name="big_object_insert">
+        <output-dir compare="Text">big_object_insert</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="big-object">
+      <compilation-unit name="big_object_load_only_20M">
+        <output-dir compare="Text">big_object_load_only_20M</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="leftouterjoin">
+    <test-case FilePath="leftouterjoin">
+      <compilation-unit name="loj-01-core">
+        <output-dir compare="Text">loj-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="leftouterjoin">
+      <compilation-unit name="loj-01-sugar">
+        <output-dir compare="Text">loj-01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="leftouterjoin">
+      <compilation-unit name="loj-02-push-select">
+        <output-dir compare="Text">loj-02-push-select</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="leftouterjoin">
+      <compilation-unit name="loj-03-no-listify">
+        <output-dir compare="Text">loj-03-no-listify</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="leftouterjoin">
+      <compilation-unit name="query_issue658">
+        <output-dir compare="Text">query_issue658</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="leftouterjoin">
+      <compilation-unit name="query_issue285">
+        <output-dir compare="Text">query_issue285</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="leftouterjoin">
+      <compilation-unit name="query_issue285-2">
+        <output-dir compare="Text">query_issue285-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="leftouterjoin">
+      <compilation-unit name="query_issue849">
+        <output-dir compare="Text">query_issue849</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="leftouterjoin">
+      <compilation-unit name="query_issue849-2">
+        <output-dir compare="Text">query_issue849-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="leftouterjoin">
+      <compilation-unit name="empty-dataset">
+        <output-dir compare="Text">empty-dataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="leftouterjoin">
+      <compilation-unit name="query-ASTERIXDB-769">
+        <output-dir compare="Text">query-ASTERIXDB-769</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="leftouterjoin">
+      <compilation-unit name="query-ASTERIXDB-2857">
+        <output-dir compare="Text">query-ASTERIXDB-2857</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="leftouterjoin">
+      <compilation-unit name="right_branch_opt_1">
+        <output-dir compare="Text">right_branch_opt_1</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="index-leftouterjoin">
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-btree-sidx1">
+        <output-dir compare="Text">probe-pidx-with-join-btree-sidx1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-btree-sidx2">
+        <output-dir compare="Text">probe-pidx-with-join-btree-sidx2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-join-btree-sidx3-idxonly">
+        <output-dir compare="Text">probe-pidx-join-btree-sidx3-idxonly</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-rtree-sidx1">
+        <output-dir compare="Text">probe-pidx-with-join-rtree-sidx1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-rtree-sidx2">
+        <output-dir compare="Text">probe-pidx-with-join-rtree-sidx2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-invidx-sidx1">
+        <output-dir compare="Text">probe-pidx-with-join-invidx-sidx1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-invidx-sidx2">
+        <output-dir compare="Text">probe-pidx-with-join-invidx-sidx2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-idxonly-join-btree-pidx1">
+        <output-dir compare="Text">probe-sidx-btree-idxonly-join-btree-pidx1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-idxonly-join-btree-sidx1-idxonly">
+        <output-dir compare="Text">probe-sidx-btree-idxonly-join-btree-sidx1-idxonly</output-dir>
+      </compilation-unit>
+    </test-case>
+    <!--test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-non-idxonly-join-btree-pidx1">
+        <output-dir compare="Text">probe-sidx-btree-non-idxonly-join-btree-pidx1</output-dir>
+      </compilation-unit>
+    </test-case!-->
+    <!--test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-sidx-btree-non-idxonly-join-btree-sidx1-idxonly">
+        <output-dir compare="Text">probe-sidx-btree-non-idxonly-join-btree-sidx1-idxonly</output-dir>
+      </compilation-unit>
+    </test-case!-->
+    <test-case FilePath="index-leftouterjoin">
+      <compilation-unit name="probe-pidx-with-join-btree-pidx1">
+        <output-dir compare="Text">probe-pidx-with-join-btree-pidx1</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="distinct">
+    <test-case FilePath="distinct">
+      <compilation-unit name="array">
+        <output-dir compare="Text">array</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="distinct">
+      <compilation-unit name="record">
+        <output-dir compare="Text">record</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="distinct">
+      <compilation-unit name="query-issue443">
+        <output-dir compare="Text">query-issue443</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="distinct">
+      <compilation-unit name="query-issue443-2">
+        <output-dir compare="Text">query-issue443-2</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="rightouterjoin">
+    <test-case FilePath="rightouterjoin">
+      <compilation-unit name="roj-01-core">
+        <output-dir compare="Text">roj-01-core</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="rightouterjoin">
+      <compilation-unit name="roj-02-core">
+        <output-dir compare="Text">roj-02-core</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="rightouterjoin">
+      <compilation-unit name="roj-03-negative">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1130: Illegal use of RIGHT OUTER JOIN</expected-error>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="tinysocial">
+    <test-case FilePath="tinysocial">
+      <compilation-unit name="tinysocial-suite">
+        <output-dir compare="Text">tinysocial-suite</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tinysocial">
+      <compilation-unit name="tinysocial-suite-open">
+        <output-dir compare="Text">tinysocial-suite-open</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="types">
+    <test-case FilePath="types">
+      <compilation-unit name="any-object">
+        <output-dir compare="Text">any-object</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="gettype">
+        <output-dir compare="Text">gettype</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="isarray">
+        <output-dir compare="Text">isarray</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="isatomic">
+        <output-dir compare="Text">isatomic</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="isboolean">
+        <output-dir compare="Text">isboolean</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="isnumber">
+        <output-dir compare="Text">isnumber</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="isobject">
+        <output-dir compare="Text">isobject</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="isstring">
+        <output-dir compare="Text">isstring</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="isbinary">
+        <output-dir compare="Text">isbinary</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="iscircle">
+        <output-dir compare="Text">iscircle</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="isdate">
+        <output-dir compare="Text">isdate</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="isdatetime">
+        <output-dir compare="Text">isdatetime</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="isduration">
+        <output-dir compare="Text">isduration</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="isinterval">
+        <output-dir compare="Text">isinterval</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="isline">
+        <output-dir compare="Text">isline</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="ismultiset">
+        <output-dir compare="Text">ismultiset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="ispoint">
+        <output-dir compare="Text">ispoint</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="ispolygon">
+        <output-dir compare="Text">ispolygon</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="isrectangle">
+        <output-dir compare="Text">isrectangle</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="isspatial">
+        <output-dir compare="Text">isspatial</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="istemporal">
+        <output-dir compare="Text">istemporal</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="istime">
+        <output-dir compare="Text">istime</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="isuuid">
+        <output-dir compare="Text">isuuid</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="is_all_types">
+        <output-dir compare="Text">is_all_types</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="record01">
+        <output-dir compare="Text">record01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="type_promotion_0">
+        <output-dir compare="Text">type_promotion_0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="type_promotion_1">
+        <output-dir compare="Text">type_promotion_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="opentype_orderby_01">
+        <output-dir compare="Text">opentype_orderby_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_closedtype_field_01">
+        <output-dir compare="Text">promotion_closedtype_field_vs_closedtype_field_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_closedtype_field_02">
+        <output-dir compare="Text">promotion_closedtype_field_vs_closedtype_field_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_closedtype_field_03">
+        <output-dir compare="Text">promotion_closedtype_field_vs_closedtype_field_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_closedtype_field_04">
+        <output-dir compare="Text">promotion_closedtype_field_vs_closedtype_field_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_closedtype_field_05">
+        <output-dir compare="Text">promotion_closedtype_field_vs_closedtype_field_05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_constant_01">
+        <output-dir compare="Text">promotion_closedtype_field_vs_constant_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_constant_02">
+        <output-dir compare="Text">promotion_closedtype_field_vs_constant_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_constant_03">
+        <output-dir compare="Text">promotion_closedtype_field_vs_constant_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_constant_04">
+        <output-dir compare="Text">promotion_closedtype_field_vs_constant_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_constant_05">
+        <output-dir compare="Text">promotion_closedtype_field_vs_constant_05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_constant_06">
+        <output-dir compare="Text">promotion_closedtype_field_vs_constant_06</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_constant_07">
+        <output-dir compare="Text">promotion_closedtype_field_vs_constant_07</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_constant_08">
+        <output-dir compare="Text">promotion_closedtype_field_vs_constant_08</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_constant_09">
+        <output-dir compare="Text">promotion_closedtype_field_vs_constant_09</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_constant_10">
+        <output-dir compare="Text">promotion_closedtype_field_vs_constant_10</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_constant_11">
+        <output-dir compare="Text">promotion_closedtype_field_vs_constant_11</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_opentype_field_01">
+        <output-dir compare="Text">promotion_closedtype_field_vs_opentype_field_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_opentype_field_02">
+        <output-dir compare="Text">promotion_closedtype_field_vs_opentype_field_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_opentype_field_03">
+        <output-dir compare="Text">promotion_closedtype_field_vs_opentype_field_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_closedtype_field_vs_opentype_field_04">
+        <output-dir compare="Text">promotion_closedtype_field_vs_opentype_field_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_opentype_field_vs_constant_01">
+        <output-dir compare="Text">promotion_opentype_field_vs_constant_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_opentype_field_vs_constant_02">
+        <output-dir compare="Text">promotion_opentype_field_vs_constant_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_opentype_field_vs_constant_03">
+        <output-dir compare="Text">promotion_opentype_field_vs_constant_03</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_opentype_field_vs_constant_04">
+        <output-dir compare="Text">promotion_opentype_field_vs_constant_04</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_opentype_field_vs_constant_05">
+        <output-dir compare="Text">promotion_opentype_field_vs_constant_05</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_opentype_field_vs_constant_06">
+        <output-dir compare="Text">promotion_opentype_field_vs_constant_06</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_opentype_field_vs_constant_07">
+        <output-dir compare="Text">promotion_opentype_field_vs_constant_07</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_opentype_field_vs_constant_08">
+        <output-dir compare="Text">promotion_opentype_field_vs_constant_08</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_opentype_field_vs_opentype_field_01">
+        <output-dir compare="Text">promotion_opentype_field_vs_opentype_field_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="promotion_opentype_field_vs_opentype_field_02">
+        <output-dir compare="Text">promotion_opentype_field_vs_opentype_field_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="to_array">
+        <output-dir compare="Text">to_array</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="to_atomic">
+        <output-dir compare="Text">to_atomic</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="to_boolean_01">
+        <output-dir compare="Text">to_boolean_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types" check-warnings="true">
+      <compilation-unit name="to_boolean_02">
+        <output-dir compare="Text">to_boolean_02</output-dir>
+        <expected-warn>ASX0004: Unsupported type: to-boolean() cannot process input type date (in line 24, at column 8)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="to_bigint_01">
+        <output-dir compare="Text">to_bigint_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types" check-warnings="true">
+      <compilation-unit name="to_bigint_02">
+        <output-dir compare="Text">to_bigint_02</output-dir>
+        <expected-warn>ASX0004: Unsupported type: to-bigint() cannot process input type date (in line 24, at column 7)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="to_double_01">
+        <output-dir compare="Text">to_double_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types" check-warnings="true">
+      <compilation-unit name="to_double_02">
+        <output-dir compare="Text">to_double_02</output-dir>
+        <expected-warn>ASX0004: Unsupported type: to-double() cannot process input type date (in line 24, at column 7)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="to_number_01">
+        <output-dir compare="Text">to_number_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="to_number_02">
+        <output-dir compare="Text">to_number_02</output-dir>
+        <expected-error>ASX0002: Type mismatch</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="to_object">
+        <output-dir compare="Text">to_object</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="to_string_01">
+        <output-dir compare="Text">to_string_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="to_string_02">
+        <output-dir compare="Text">to_string_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="domain_boundaries">
+        <output-dir compare="Text">domain_boundaries</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="domain_boundaries_error">
+        <output-dir compare="Text">domain_boundaries_error</output-dir>
+        <expected-error>ASX0001: Field type null cannot be promoted to type tinyint</expected-error>
+        <expected-error>ASX0001: Field type null cannot be promoted to type smallint</expected-error>
+        <expected-error>ASX0001: Field type null cannot be promoted to type integer</expected-error>
+        <expected-error>ASX0001: Field type null cannot be promoted to type bigint</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="types">
+      <compilation-unit name="query-ASTERIXDB-2950">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX1002: Type mismatch: function get-item expects its 1st input parameter to be of type multiset or array, but the actual input type is bigint (in line 24, at column 21)</expected-error>
+        <expected-error>ASX1002: Type mismatch: function get-item expects its 1st input parameter to be of type multiset or array, but the actual input type is bigint (in line 25, at column 31)</expected-error>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="cleanjson">
+    <test-case FilePath="json">
+      <compilation-unit name="issue-ASTERIXDB-1165">
+        <output-dir compare="Clean-JSON">issue-ASTERIXDB-1165</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="json">
+      <compilation-unit name="int01">
+        <output-dir compare="Clean-JSON">int01-cleanjson</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="materialization">
+    <test-case FilePath="materialization">
+      <compilation-unit name="assign-reuse">
+        <output-dir compare="Text">assign-reuse</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="filters">
+    <test-case FilePath="filters">
+      <compilation-unit name="equality-predicate">
+        <output-dir compare="Text">equality-predicate</output-dir>
+      </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>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="load-with-secondary-btree">
+        <output-dir compare="Text">load-with-secondary-btree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="load-with-secondary-btree-index-only">
+        <output-dir compare="Text">load-with-secondary-btree-index-only</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="load-with-secondary-inverted-ngram">
+        <output-dir compare="Text">load-with-secondary-inverted-ngram</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="load-with-secondary-inverted-word">
+        <output-dir compare="Text">load-with-secondary-inverted-word</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="load-with-secondary-rtree">
+        <output-dir compare="Text">load-with-secondary-rtree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="insert">
+        <output-dir compare="Text">insert</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="insert-with-secondary-btree">
+        <output-dir compare="Text">insert-with-secondary-btree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="insert-with-correlated-secondary-btree">
+        <output-dir compare="Text">insert-with-secondary-btree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="insert-with-secondary-inverted-ngram">
+        <output-dir compare="Text">insert-with-secondary-inverted-ngram</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="insert-with-correlated-secondary-inverted-ngram">
+        <output-dir compare="Text">insert-with-secondary-inverted-ngram</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="insert-with-secondary-inverted-word">
+        <output-dir compare="Text">insert-with-secondary-inverted-word</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="insert-with-correlated-secondary-inverted-word">
+        <output-dir compare="Text">insert-with-secondary-inverted-word</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="insert-with-secondary-rtree">
+        <output-dir compare="Text">insert-with-secondary-rtree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="insert-with-correlated-secondary-rtree">
+        <output-dir compare="Text">insert-with-secondary-rtree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="nested-filter-equality-predicate">
+        <output-dir compare="Text">nested-filter-equality-predicate</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="upsert">
+        <output-dir compare="Text">upsert</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="filters">
+      <compilation-unit name="delete">
+        <output-dir compare="Text">delete</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="json">
+    <test-case FilePath="json">
+      <compilation-unit name="int01">
+        <output-dir compare="Lossless-JSON">int01-losslessjson</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="csv">
+    <test-case FilePath="csv">
+      <compilation-unit name="basic-types">
+        <output-dir compare="CSV">basic-types</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="csv">
+      <compilation-unit name="basic-types">
+        <output-dir compare="CSV_Header">basic-types-header</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="csv-tsv-parser">
+    <test-case FilePath="csv-tsv-parser">
+      <compilation-unit name="csv-parser-001">
+        <output-dir compare="Text">csv-parser-001</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="csv-tsv-parser">
+      <compilation-unit name="tsv-parser-001">
+        <output-dir compare="Text">tsv-parser-001</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="binary">
+    <test-case FilePath="binary">
+      <compilation-unit name="parse">
+        <output-dir compare="Text">parse</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="binary">
+      <compilation-unit name="print">
+        <output-dir compare="Text">print</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="binary">
+      <compilation-unit name="concat">
+        <output-dir compare="Text">concat</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="binary">
+      <compilation-unit name="concat2">
+        <output-dir compare="Text">concat2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="binary">
+      <compilation-unit name="subbinary">
+        <output-dir compare="Text">subbinary</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="binary">
+      <compilation-unit name="find">
+        <output-dir compare="Text">find</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="binary">
+      <compilation-unit name="insert">
+        <output-dir compare="Text">insert</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="binary">
+      <compilation-unit name="equal_join">
+        <output-dir compare="Text">equal_join</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="binary">
+      <compilation-unit name="index_join">
+        <output-dir compare="Text">index_join</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="binary">
+      <compilation-unit name="length">
+        <output-dir compare="Text">length</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="unnest">
+    <test-case FilePath="unnest">
+      <compilation-unit name="ASTERIXDB-2750_unnest_join">
+        <output-dir compare="Text">ASTERIXDB-2750_unnest_join</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="unnest">
+      <compilation-unit name="ASTERIXDB-2844_unnest_syntax">
+        <output-dir compare="Text">ASTERIXDB-2844_unnest_syntax</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="unnest">
+      <compilation-unit name="left-outer-unnest">
+        <output-dir compare="Text">left-outer-unnest</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="unnest">
+      <compilation-unit name="left-outer-unnest-with-pos">
+        <output-dir compare="Text">left-outer-unnest-with-pos</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="union">
+    <test-case FilePath="union">
+      <compilation-unit name="union">
+        <output-dir compare="Text">union</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="union_heterogeneous_scalar">
+        <output-dir compare="Text">union_heterogeneous_scalar</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="union_negative">
+        <output-dir compare="Text">union</output-dir>
+        <expected-error>Cannot find dataset t in dataverse TinySocial nor an alias with name t</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="union_negative_3">
+        <output-dir compare="Text">union</output-dir>
+        <expected-error>Operation UNION with set semantics is not supported.</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="union_nested">
+        <output-dir compare="Text">union_nested</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="union_opt_1">
+        <output-dir compare="Text">union_opt_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="union_orderby">
+        <output-dir compare="Text">union_orderby</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="union_orderby_2">
+        <output-dir compare="Text">union_orderby_2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="union_orderby_3">
+        <output-dir compare="Text">union_orderby_3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="union_orderby_4">
+        <output-dir compare="Text">union_orderby_3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="union_orderby_5">
+        <output-dir compare="Text">union_orderby_5</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="query-ASTERIXDB-1354-2">
+        <output-dir compare="Text">query-ASTERIXDB-1354-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="query-ASTERIXDB-1629">
+        <output-dir compare="Text">query-ASTERIXDB-1629</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="query-ASTERIXDB-1047">
+        <output-dir compare="Text">query-ASTERIXDB-1047</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="query-ASTERIXDB-1205-2">
+        <output-dir compare="Text">query-ASTERIXDB-1205-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="query-ASTERIXDB-1205-3">
+        <output-dir compare="Text">query-ASTERIXDB-1205-3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="query-ASTERIXDB-1205">
+        <output-dir compare="Text">query-ASTERIXDB-1205</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="query-ASTERIXDB-1354">
+        <output-dir compare="Text">query-ASTERIXDB-1354</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="upsert">
+    <test-case FilePath="upsert">
+      <compilation-unit name="filtered-dataset">
+        <output-dir compare="Text">filtered-dataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="issue1587-foreignDataType">
+        <output-dir compare="Text">issue1587-foreignDataType</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="nested-index">
+        <output-dir compare="Text">nested-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="primary-secondary-rtree">
+        <output-dir compare="Text">primary-secondary-rtree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="primary-correlated-secondary-rtree">
+        <output-dir compare="Text">primary-secondary-rtree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="upsert-with-self-read">
+        <output-dir compare="Text">upsert-with-self-read</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="nullable-index">
+        <output-dir compare="Text">nullable-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="open-index">
+        <output-dir compare="Text">open-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="primary-index">
+        <output-dir compare="Text">primary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="primary-secondary-btree">
+        <output-dir compare="Text">primary-secondary-btree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="primary-correlated-secondary-btree">
+        <output-dir compare="Text">primary-secondary-btree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="primary-secondary-inverted">
+        <output-dir compare="Text">primary-secondary-inverted</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="primary-correlated-secondary-inverted">
+        <output-dir compare="Text">primary-secondary-inverted</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="multiple-secondaries">
+        <output-dir compare="Text">multiple-secondaries</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="multiple-correlated-secondaries">
+        <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">
+      <compilation-unit name="numeric-tinyint">
+        <output-dir compare="Text">numeric-tinyint</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="json-parser">
+      <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">
+      <compilation-unit name="numeric-float">
+        <output-dir compare="Text">numeric-float</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="json-parser">
+      <compilation-unit name="numeric-demote-double-bigint">
+        <output-dir compare="Text">numeric-demote-double-bigint</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="json-parser">
+      <compilation-unit name="numeric-promote-bigint-double">
+        <output-dir compare="Text">numeric-promote-bigint-double</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="json-parser">
+      <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">
+      <compilation-unit name="null-missing">
+        <output-dir compare="Text">null-missing</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="json-parser">
+      <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">
+      <compilation-unit name="spatial">
+        <output-dir compare="Text">spatial</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="json-parser">
+      <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">
+      <compilation-unit name="temporal">
+        <output-dir compare="Text">temporal</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="json-parser">
+      <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-case FilePath="json-parser" check-warnings="true">
+      <compilation-unit name="parse-json-function">
+        <output-dir compare="Text">parse-json-function</output-dir>
+        <source-location>false</source-location>
+        <expected-warn>Type mismatch: function parse-json expects its 1st input parameter to be of type string, but the actual input type is bigint</expected-warn>
+        <expected-warn>Malformed input stream</expected-warn>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="composite-key">
+    <test-case FilePath="composite-key">
+      <compilation-unit name="query-ASTERIXDB-920">
+        <output-dir compare="Text">query-ASTERIXDB-920</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="composite-key">
+      <compilation-unit name="query-ASTERIXDB-2334">
+        <output-dir compare="Text">query-ASTERIXDB-2334</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="composite-key">
+      <compilation-unit name="composite-low-high">
+        <output-dir compare="Text">composite-low-high</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="composite-key">
+      <compilation-unit name="composite-prefix">
+        <output-dir compare="Text">composite-prefix</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="composite-key">
+      <compilation-unit name="composite-prefix-low-high">
+        <output-dir compare="Text">composite-prefix-low-high</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="limit">
+    <test-case FilePath="limit">
+      <compilation-unit name="limit_negative_value">
+        <output-dir compare="Text">limit_negative_value</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+      <compilation-unit name="limit_type_01">
+        <output-dir compare="Text">limit_type_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+      <compilation-unit name="limit_type_02">
+        <output-dir compare="Text">limit_type_01</output-dir>
+        <expected-error>ASX0039: Expected integer value, got 2.75 (in line 28, at column 10)</expected-error>
+        <expected-error>ASX0039: Expected integer value, got 1.75 (in line 28, at column 19)</expected-error>
+        <expected-error>ASX1091: Type mismatch: expected value of type integer, but got the value of type string (in line 28, at column 7)</expected-error>
+        <expected-error>ASX1091: Type mismatch: expected value of type integer, but got the value of type boolean (in line 28, at column 16)</expected-error>
+        <expected-error>ASX0021: Source value 9999999999 is out of range that integer can hold - integer.MAX_VALUE: 2147483647, integer.MIN_VALUE: -2147483648</expected-error>
+        <expected-error>ASX0021: Source value 8888888888 is out of range that integer can hold - integer.MAX_VALUE: 2147483647, integer.MIN_VALUE: -2147483648</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+      <compilation-unit name="offset_without_limit">
+        <output-dir compare="Text">offset_without_limit</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+      <compilation-unit name="push-limit-to-external-scan">
+        <output-dir compare="Text">push-limit-to-external-scan</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+      <compilation-unit name="push-limit-to-external-scan-select">
+        <output-dir compare="Text">push-limit-to-external-scan-select</output-dir>
+      </compilation-unit>
+    </test-case>
+    <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-case FilePath="limit">
+      <compilation-unit name="query-ASTERIXDB-2420">
+        <output-dir compare="Text">query-ASTERIXDB-2420</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="compression">
+    <test-case FilePath="compression">
+      <compilation-unit name="incompressible-pages/large-page">
+        <output-dir compare="Text">incompressible-pages/large-page</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="compression">
+      <compilation-unit name="incompressible-pages/small-page">
+        <output-dir compare="Text">incompressible-pages/small-page</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="compression">
+      <compilation-unit name="invalid-compression-scheme">
+        <output-dir compare="Text">invalid-compression-scheme</output-dir>
+        <expected-error>ASX1096: Unknown compression scheme zip. Supported schemes are [snappy,none]</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="compression">
+      <compilation-unit name="scheme-none">
+        <output-dir compare="Text">scheme-none</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="compression">
+      <compilation-unit name="scheme-snappy">
+        <output-dir compare="Text">scheme-snappy</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="ddl-with-clause">
+    <test-case FilePath="ddl-with-clause">
+      <compilation-unit name="missing-non-optional">
+        <output-dir compare="Text">missing-non-optional</output-dir>
+        <expected-error>ASX1061: Field 'merge-policy.name' in the with clause cannot be null or missing</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl-with-clause">
+      <compilation-unit name="type-mismatch">
+        <output-dir compare="Text">type-mismatch</output-dir>
+        <expected-error>ASX1060: Field 'merge-policy.parameters.max-mergable-component-size' in the with clause must be of type bigint, but found string</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl-with-clause">
+      <compilation-unit name="unsupported-field">
+        <output-dir compare="Text">unsupported-field</output-dir>
+        <expected-error>ASX1059: Field(s) [unknown-field] unsupported in the with clause</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="ddl-with-clause">
+      <compilation-unit name="unsupported-subfield">
+        <output-dir compare="Text">unsupported-subfield</output-dir>
+        <expected-error>ASX1097: Subfield(s) [unknown-subfield] in 'merge-policy' unsupported in the with clause</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="metrics">
+    <test-case FilePath="metrics">
+      <compilation-unit name="full-scan">
+        <output-dir compare="Text">full-scan</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="warnings">
+    <test-case FilePath="warnings" check-warnings="true">
+      <compilation-unit name="inapplicable-hint-warning">
+        <output-dir compare="Text">inapplicable-hint-warning</output-dir>
+        <expected-warn>HYR10006: Could not apply Group By hint: hash</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: indexnl. "hash" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: hash. "hash-bcast", "indexnl", "hashjoin", "skip-index", "use-index", "selectivity", "productivity" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: auto. "indexnl", "skip-index", "hashjoin", "use-index", "selectivity" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: hash. "indexnl", "range", "hashjoin", "skip-index", "spatial-partitioning", "use-index" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: hash. None expected at this location</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="warnings" check-warnings="true">
+      <compilation-unit name="min-max-incompatible-types">
+        <output-dir compare="Text">min-max-incompatible-types</output-dir>
+        <expected-warn>ASX0003: Type incompatibility: function min/max gets incompatible input values: bigint and string</expected-warn>
+        <expected-warn>ASX0004: Unsupported type: min/max cannot process input type object</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="warnings" check-warnings="true">
+      <compilation-unit name="plan-warning">
+        <output-dir compare="Text">plan-warning</output-dir>
+        <expected-warn>HYR10007: Encountered a cross product join</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="warnings" check-warnings="true">
+      <compilation-unit name="unknown-hint-warning">
+        <output-dir compare="Text">unknown-hint-warning</output-dir>
+        <expected-warn>ASX1107: Unexpected hint: unknown_hint_groupby. "hash" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr. "hash-bcast", "indexnl", "hashjoin", "skip-index", "use-index", "selectivity", "productivity" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: unknown_hint_between. "indexnl", "skip-index", "hashjoin", "use-index", "selectivity" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: unknown_hint_funcall. "indexnl", "range", "hashjoin", "skip-index", "spatial-partitioning", "use-index" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: unknown_hint_elsewhere. None expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr_6. "hash-bcast", "indexnl", "hashjoin", "skip-index", "use-index", "selectivity", "productivity" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr_6. "hash-bcast", "indexnl", "hashjoin", "skip-index", "use-index", "selectivity", "productivity" expected at this location</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <!--test-case FilePath="warnings">
+      <compilation-unit name="warnings-limit">
+        <output-dir compare="Clean-JSON">warnings-limit</output-dir>
+      </compilation-unit>
+    </test-case!-->
+  </test-group>
+  <test-group name="nonpure">
+    <test-case FilePath="nonpure">
+      <compilation-unit name="global-datetime-use-index">
+        <output-dir compare="Text">global-datetime-use-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="nonpure">
+      <compilation-unit name="local-datetime-ignore-index">
+        <output-dir compare="Text">local-datetime-ignore-index</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="orderby_limit">
+    <test-case FilePath="orderby_limit">
+      <compilation-unit name="limit_on_variable_01">
+        <output-dir compare="Text">limit_on_variable_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="orderby_limit">
+      <compilation-unit name="orderby_limit_01">
+        <output-dir compare="Text">orderby_limit_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="orderby_limit">
+      <compilation-unit name="orderby_limit_02">
+        <output-dir compare="Text">orderby_limit_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="orderby_limit">
+      <compilation-unit name="orderby_limit_offset_01">
+        <output-dir compare="Text">orderby_limit_offset_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="orderby_limit">
+      <compilation-unit name="orderby_limit_primary_index_01">
+        <output-dir compare="Text">orderby_limit_primary_index_01</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="fun_return_null_missing/string_fun">
+    <test-case FilePath="fun_return_null_missing/string_fun" check-warnings="true">
+      <compilation-unit name="string_fun_001">
+        <output-dir compare="Text">string_fun_001</output-dir>
+        <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 41, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function like expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 32, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function uppercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 38, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function ltrim expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 42, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function like expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 33, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-length expects its 1st input parameter to be of type string, but the actual input type is integer (in line 36, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 40, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function lowercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 37, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 45, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function contains expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 34, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function ltrim expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 43, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function position expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 46, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-to-codepoint expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 35, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function initcap expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 39, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 44, at column 1)</expected-warn>
+
+        <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 41, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function like expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 32, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function uppercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 38, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function ltrim expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 42, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function like expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 33, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-length expects its 1st input parameter to be of type string, but the actual input type is integer (in line 36, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 40, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function lowercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 37, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 45, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function contains expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 34, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function ltrim expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 43, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function position expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 46, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-to-codepoint expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 35, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function initcap expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 39, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 44, at column 1)</expected-warn>
+
+        <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 41, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function like expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function uppercase expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 38, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function ltrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 42, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function like expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 33, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-length expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 36, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 40, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function lowercase expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 37, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 45, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function contains expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 34, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function ltrim expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 43, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function position expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 46, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-to-codepoint expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 35, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function initcap expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 39, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 44, at column 1)</expected-warn>
+
+        <expected-warn>Type mismatch: function regexp-position expects its 1st input parameter to be of type string, but the actual input type is integer (in line 38, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function replace expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 42, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function starts-with expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function reverse expects its 1st input parameter to be of type string, but the actual input type is integer (in line 43, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function regexp-like expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 37, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-equal expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 41, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function substring-after expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 44, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function matches expects its 2nd input parameter to be of type string, but the actual input type is integer (in line 34, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function regexp-like expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 36, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function regexp-position expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 39, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function substring-before expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 45, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function regexp-replace expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 40, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function matches expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 35, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function split expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 46, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function ends-with expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 33, at column 1)</expected-warn>
+
+        <expected-warn>Type mismatch: function regexp-position expects its 1st input parameter to be of type string, but the actual input type is integer (in line 38, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function replace expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 42, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function starts-with expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function reverse expects its 1st input parameter to be of type string, but the actual input type is integer (in line 43, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function regexp-like expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 37, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-equal expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 41, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function substring-after expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 44, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function matches expects its 2nd input parameter to be of type string, but the actual input type is integer (in line 34, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function regexp-like expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 36, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function regexp-position expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 39, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function substring-before expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 45, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function regexp-replace expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 40, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function matches expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 35, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function split expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 46, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function ends-with expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 33, at column 1)</expected-warn>
+
+        <expected-warn>Type mismatch: function regexp-position expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 38, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function replace expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 42, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function starts-with expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function reverse expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 43, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function regexp-like expects its 3rd input parameter to be of type string, but the actual input type is bigint (in line 37, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-equal expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 41, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function substring-after expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 44, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function matches expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 34, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function regexp-like expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 36, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function regexp-position expects its 3rd input parameter to be of type string, but the actual input type is bigint (in line 39, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function substring-before expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 45, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function regexp-replace expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 40, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function matches expects its 3rd input parameter to be of type string, but the actual input type is bigint (in line 35, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function split expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 46, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function ends-with expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 33, at column 1)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fun_return_null_missing/string_fun" check-warnings="true">
+      <compilation-unit name="string_fun_002">
+        <output-dir compare="Text">string_fun_002</output-dir>
+        <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 42, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function regexp-replace expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 52, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function regexp-like expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 48, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function position expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 43, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function matches expects its 2nd input parameter to be of type string, but the actual input type is integer (in line 46, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function replace expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 54, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function starts-with expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 44, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function regexp-position expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 51, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function rtrim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 41, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function substring-after expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 56, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function substring-before expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 57, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function lowercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 34, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function ends-with expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 45, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 38, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function uppercase expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 35, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function regexp-like expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 49, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function matches expects its 3rd input parameter to be of type string, but the actual input type is tinyint (in line 47, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-equal expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 53, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function split expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 58, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-to-codepoint expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function trim expects its 1st input parameter to be of type string, but the actual input type is integer (in line 37, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function reverse expects its 1st input parameter to be of type string, but the actual input type is integer (in line 55, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function like expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 29, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function contains expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 31, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function ltrim expects its 1st input parameter to be of type string, but the actual input type is smallint (in line 39, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function initcap expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 36, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function ltrim expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 40, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-length expects its 1st input parameter to be of type string, but the actual input type is integer (in line 33, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function like expects its 1st input parameter to be of type string, but the actual input type is tinyint (in line 30, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function regexp-position expects its 1st input parameter to be of type string, but the actual input type is integer (in line 50, at column 1)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fun_return_null_missing/string_fun" check-warnings="true">
+      <compilation-unit name="string_fun_003">
+        <output-dir compare="Text">string_fun_003</output-dir>
+        <expected-warn>Invalid value: function repeat expects its 2nd input parameter to be an integer value, got 5.3 (in line 31, at column 1)</expected-warn>
+        <expected-warn>Invalid value: function regexp-replace expects its 4th input parameter to be an integer value, got 5.3 (in line 36, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function substring expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 37, at column 1)</expected-warn>
+        <expected-warn>Invalid value: function replace expects its 4th input parameter to be an integer value, got 5.3 (in line 34, at column 1)</expected-warn>
+        <expected-warn>Invalid value: function repeat expects its 2nd input parameter to be a non-negative value, got -2.0 (in line 30, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
+
+        <expected-warn>Invalid value: function repeat expects its 2nd input parameter to be an integer value, got 5.3 (in line 31, at column 1)</expected-warn>
+        <expected-warn>Invalid value: function regexp-replace expects its 4th input parameter to be an integer value, got 5.3 (in line 36, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function substring expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 37, at column 1)</expected-warn>
+        <expected-warn>Invalid value: function replace expects its 4th input parameter to be an integer value, got 5.3 (in line 34, at column 1)</expected-warn>
+        <expected-warn>Invalid value: function repeat expects its 2nd input parameter to be a non-negative value, got -2.0 (in line 30, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-join expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 32, at column 1)</expected-warn>
+
+        <expected-warn>Invalid value: function repeat expects its 2nd input parameter to be an integer value, got Infinity (in line 29, at column 1)</expected-warn>
+        <expected-warn>Invalid value: function replace expects its 4th input parameter to be an integer value, got -Infinity (in line 30, at column 1)</expected-warn>
+        <expected-warn>Invalid value: function replace expects its 4th input parameter to be an integer value, got NaN (in line 31, at column 1)</expected-warn>
+        <expected-warn>Invalid value: function regexp-replace expects its 4th input parameter to be an integer value, got NaN (in line 32, at column 1)</expected-warn>
+        <expected-warn>Invalid value: function regexp-replace expects its 4th input parameter to be an integer value, got -Infinity (in line 33, at column 1)</expected-warn>
+        <expected-warn>Invalid value: function substring expects its 2nd input parameter to be an integer value, got Infinity (in line 34, at column 1)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fun_return_null_missing/string_fun" check-warnings="true">
+      <compilation-unit name="string_fun_004">
+        <output-dir compare="Text">string_fun_004</output-dir>
+        <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 30, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function substring expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 32, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function substring expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 33, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 31, at column 7)</expected-warn>
+        <expected-warn>Type mismatch: function codepoint-to-string expects its 1st input parameter to be of type array, but the actual input type is string (in line 34, at column 1)</expected-warn>
+        <expected-warn>Unsupported type: codepoint-to-string cannot process input type string (in line 35, at column 1)</expected-warn>
+
+        <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 30, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function substring expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 32, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function substring expects its 1st input parameter to be of type string, but the actual input type is bigint (in line 33, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 31, at column 7)</expected-warn>
+        <expected-warn>Type mismatch: function codepoint-to-string expects its 1st input parameter to be of type array, but the actual input type is string (in line 34, at column 1)</expected-warn>
+        <expected-warn>Unsupported type: codepoint-to-string cannot process input type string (in line 35, at column 1)</expected-warn>
+
+        <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is bigint (in line 30, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function string-concat expects its 2nd input parameter to be of type string, but the actual input type is tinyint (in line 31, at column 7)</expected-warn>
+        <expected-warn>Type mismatch: function substring expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 32, at column 1)</expected-warn>
+        <expected-warn>Type mismatch: function codepoint-to-string expects its 1st input parameter to be of type array, but the actual input type is string (in line 33, at column 1)</expected-warn>
+        <expected-warn>Unsupported type: codepoint-to-string cannot process input type string (in line 34, at column 1)</expected-warn>
+        <expected-warn>Unsupported type: codepoint-to-string cannot process input type double (in line 35, at column 1)</expected-warn>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="fun_return_null_missing/numeric_fun" >
+    <test-case FilePath="fun_return_null_missing/numeric_fun" check-warnings="true">
+      <compilation-unit name="numeric_fun_001">
+        <output-dir compare="Text">numeric_fun_001</output-dir>
+        <expected-warn>Type mismatch: function abs expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function acos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string </expected-warn>
+        <expected-warn>Type mismatch: function asin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function atan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function atan2 expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function degrees expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function radians expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function cos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function cosh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function sin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function sinh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function tan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function tanh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function exp expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function ln expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function log expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function sqrt expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function sign expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function ceiling expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function floor expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-unary-minus expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function random expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+
+        <expected-warn>Type mismatch: function abs expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function acos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string </expected-warn>
+        <expected-warn>Type mismatch: function asin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function atan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function atan2 expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function degrees expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function radians expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function cos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function cosh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function sin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function sinh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function tan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function tanh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function exp expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function ln expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function log expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function sqrt expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function sign expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function ceiling expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function floor expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-unary-minus expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function random expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+
+        <expected-warn>Type mismatch: function abs expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function acos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string </expected-warn>
+        <expected-warn>Type mismatch: function asin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function atan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function atan2 expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function degrees expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function radians expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function cos expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function cosh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function sin expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function sinh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function tan expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function tanh expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function exp expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function ln expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function log expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function sqrt expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function sign expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function ceiling expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function floor expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-unary-minus expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function random expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fun_return_null_missing/numeric_fun" check-warnings="true">
+      <compilation-unit name="numeric_fun_002">
+        <output-dir compare="Text">numeric_fun_002</output-dir>
+        <expected-warn>Type mismatch: function round expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function round-half-to-even expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function trunc expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function trunc expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function round expects its 2nd input parameter to be of type tinyint, smallint, integer or bigint, but the actual input type is string</expected-warn>
+
+        <expected-warn>Type mismatch: function round expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function round-half-to-even expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function trunc expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function round expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function round-half-to-even expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function trunc expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string</expected-warn>
+
+        <expected-warn>Type mismatch: function round expects its 2nd input parameter to be of type tinyint, smallint, integer or bigint, but the actual input type is string (in line 33, at column 24)</expected-warn>
+        <expected-warn>Type mismatch: function round expects its 2nd input parameter to be of type tinyint, smallint, integer or bigint, but the actual input type is string (in line 33, at column 47)</expected-warn>
+        <expected-warn>Type mismatch: function trunc expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 34, at column 24)</expected-warn>
+        <expected-warn>Type mismatch: function trunc expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 34, at column 47)</expected-warn>
+
+        <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 36, at column 40)</expected-warn>
+        <expected-warn>Type mismatch: function round-half-to-even expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float or double, but the actual input type is string (in line 36, at column 79)</expected-warn>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fun_return_null_missing/numeric_fun" check-warnings="true">
+      <compilation-unit name="numeric_fun_003">
+        <output-dir compare="Text">numeric_fun_003</output-dir>
+        <expected-warn>Type mismatch: function numeric-add expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-divide expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type incompatibility: function numeric-multiply gets incompatible input values: daytimeduration and string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-add expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type incompatibility: function numeric-divide gets incompatible input values: time and string</expected-warn>
+        <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: yearmonthduration and string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-multiply expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: daytimeduration and string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-div expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-subtract expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function power expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-mod expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: time and string</expected-warn>
+        <expected-warn>Type incompatibility: function power gets incompatible input values: yearmonthduration and string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-multiply expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-multiply expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-mod expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-divide expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-div expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type incompatibility: function numeric-multiply gets incompatible input values: date and string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-add expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: duration and string</expected-warn>
+        <expected-warn>Type incompatibility: function numeric-div gets incompatible input values: datetime and string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-mod expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function power expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: date and string</expected-warn>
+        <expected-warn>Type incompatibility: function numeric-mod gets incompatible input values: duration and string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-subtract expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-divide expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-div expects its 2nd input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function numeric-subtract expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <expected-warn>Type incompatibility: function numeric-add gets incompatible input values: datetime and string</expected-warn>
+        <expected-warn>Type mismatch: function power expects its 1st input parameter to be of type tinyint, smallint, integer, bigint, float, double, date, time, datetime, duration, yearmonthduration or daytimeduration, but the actual input type is string</expected-warn>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="fun_return_null_missing" check-warnings="true">
+      <compilation-unit name="field-access">
+        <output-dir compare="Text">field-access</output-dir>
+        <expected-warn>Type mismatch: function field-access-by-name expects its 1st input parameter to be of type object, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function field-access-by-name expects its 1st input parameter to be of type object, but the actual input type is string</expected-warn>
+        <expected-warn>Type mismatch: function field-access-by-name expects its 1st input parameter to be of type object, but the actual input type is string</expected-warn>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+  <test-group name="window">
+    <test-case FilePath="window">
+      <compilation-unit name="cume_dist_01">
+        <output-dir compare="Text">cume_dist_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="dense_rank_01">
+        <output-dir compare="Text">dense_rank_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="first_value_01">
+        <output-dir compare="Text">first_value_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="lag_01">
+        <output-dir compare="Text">lag_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="last_value_01">
+        <output-dir compare="Text">last_value_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="lead_01">
+        <output-dir compare="Text">lead_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="misc_01">
+        <output-dir compare="Text">misc_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="nth_value_01">
+        <output-dir compare="Text">nth_value_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="ntile_01">
+        <output-dir compare="Text">ntile_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="percent_rank_01">
+        <output-dir compare="Text">percent_rank_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="pg_win">
+        <output-dir compare="Text">pg_win</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="rank_01">
+        <output-dir compare="Text">rank_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="ratio_to_report_01">
+        <output-dir compare="Text">ratio_to_report_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="row_number_01">
+        <output-dir compare="Text">row_number_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="win_negative">
+        <output-dir compare="Text">misc_01</output-dir>
+        <expected-error>ASX0002: Type mismatch</expected-error>
+        <expected-error>ASX1104: Invalid modifier FROM FIRST/LAST for function</expected-error>
+        <expected-error>ASX1037: Invalid query parameter compiler.windowmemory</expected-error>
+        <expected-error>ASX1102: Expected window or aggregate function, got: lowercase</expected-error>
+        <expected-error>ASX1079: Compilation error: count is a SQL-92 aggregate function</expected-error>
+        <expected-error>ASX1104: Invalid modifier RESPECT/IGNORE NULLS for function</expected-error>
+        <expected-error>ASX1104: Invalid modifier RESPECT/IGNORE NULLS for function</expected-error>
+        <expected-error>ASX1104: Invalid modifier FROM FIRST/LAST for function</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="win_null_missing">
+        <output-dir compare="Text">win_null_missing</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="win_opt_01">
+        <output-dir compare="Text">win_opt_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="window">
+      <compilation-unit name="win_opt_02">
+        <output-dir compare="Text">win_opt_02</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+</test-suite>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
index 67f8253..f4e241c 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
@@ -31,6 +31,7 @@
 import org.apache.hyracks.api.application.IServiceContext;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.result.IResultSet;
 
 public interface IApplicationContext {
 
@@ -67,6 +68,11 @@
     IHyracksClientConnection getHcc() throws HyracksDataException;
 
     /**
+     * @return a result set provider associated with {@link IHyracksClientConnection}
+     */
+    IResultSet getResultSet() throws HyracksDataException;
+
+    /**
      * @return the cluster coordination service.
      */
     ICoordinationService getCoordinationService();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
index 5538a0a..62691a0 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
@@ -23,6 +23,7 @@
 import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER_BYTE_UNIT;
 import static org.apache.hyracks.control.common.config.OptionTypes.LONG_BYTE_UNIT;
 import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
+import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
 
@@ -103,7 +104,21 @@
         COMPILER_ARRAYINDEX(
                 BOOLEAN,
                 AlgebricksConfig.ARRAY_INDEX_DEFAULT,
-                "Enable/disable using array-indexes in queries");
+                "Enable/disable using array-indexes in queries"),
+        COMPILER_BATCH_LOOKUP(
+                BOOLEAN,
+                AlgebricksConfig.BATCH_LOOKUP_DEFAULT,
+                "Enable/disable batch point-lookups when running queries with secondary indexes"),
+        COMPILER_CBO(BOOLEAN, AlgebricksConfig.CBO_DEFAULT, "Set the mode for cost based optimization"),
+        COMPILER_CBOTEST(BOOLEAN, AlgebricksConfig.CBO_TEST_DEFAULT, "Set the mode for cost based optimization"),
+        COMPILER_FORCEJOINORDER(
+                BOOLEAN,
+                AlgebricksConfig.FORCE_JOIN_ORDER_DEFAULT,
+                "Set the mode for forcing the join order in a query plan"),
+        COMPILER_QUERYPLANSHAPE(
+                STRING,
+                AlgebricksConfig.QUERY_PLAN_SHAPE_DEFAULT,
+                "Set the mode for forcing the shape of the query plan");
 
         private final IOptionType type;
         private final Object defaultValue;
@@ -173,6 +188,16 @@
 
     public static final String COMPILER_EXTERNALSCANMEMORY_KEY = Option.COMPILER_EXTERNALSCANMEMORY.ini();
 
+    public static final String COMPILER_BATCHED_LOOKUP_KEY = Option.COMPILER_BATCH_LOOKUP.ini();
+
+    public static final String COMPILER_CBO_KEY = Option.COMPILER_CBO.ini();
+
+    public static final String COMPILER_CBO_TEST_KEY = Option.COMPILER_CBOTEST.ini();
+
+    public static final String COMPILER_FORCE_JOIN_ORDER_KEY = Option.COMPILER_FORCEJOINORDER.ini();
+
+    public static final String COMPILER_QUERY_PLAN_SHAPE_KEY = Option.COMPILER_QUERYPLANSHAPE.ini();
+
     public static final int COMPILER_PARALLELISM_AS_STORAGE = 0;
 
     public CompilerProperties(PropertiesAccessor accessor) {
@@ -246,4 +271,29 @@
     public int getExternalScanMemorySize() {
         return accessor.getInt(Option.COMPILER_EXTERNALSCANMEMORY);
     }
+
+    public boolean isBatchLookup() {
+        return accessor.getBoolean(Option.COMPILER_BATCH_LOOKUP);
+    }
+
+    public boolean getCBOMode() {
+        return accessor.getBoolean(Option.COMPILER_CBO);
+    }
+
+    public boolean getCBOTestMode() {
+        return accessor.getBoolean(Option.COMPILER_CBOTEST);
+    }
+
+    public boolean getForceJoinOrderMode() {
+        return accessor.getBoolean(Option.COMPILER_FORCEJOINORDER);
+    }
+
+    public String getQueryPlanShapeMode() {
+        String queryPlanShapeMode = accessor.getString(Option.COMPILER_QUERYPLANSHAPE);
+        if (!(queryPlanShapeMode.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_ZIGZAG)
+                || queryPlanShapeMode.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_LEFTDEEP)
+                || queryPlanShapeMode.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_RIGHTDEEP)))
+            return AlgebricksConfig.QUERY_PLAN_SHAPE_DEFAULT;
+        return queryPlanShapeMode;
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
index b85c0be..478bd46 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
@@ -41,7 +41,8 @@
         SINGLE_PARTITION_NGRAM_INVIX,
         LENGTH_PARTITIONED_WORD_INVIX,
         LENGTH_PARTITIONED_NGRAM_INVIX,
-        ARRAY;
+        ARRAY,
+        SAMPLE;
     }
 
     public enum TransactionState {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
index d498d69..e1edc1c 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
@@ -77,6 +77,16 @@
         int externalScanBufferSize = getExternalScanBufferSize(
                 (String) querySpecificConfig.get(CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY),
                 compilerProperties.getExternalScanMemorySize(), sourceLoc);
+        boolean batchLookup = getBoolean(querySpecificConfig, CompilerProperties.COMPILER_BATCHED_LOOKUP_KEY,
+                compilerProperties.isBatchLookup());
+        boolean cbo =
+                getBoolean(querySpecificConfig, CompilerProperties.COMPILER_CBO_KEY, compilerProperties.getCBOMode());
+        boolean cboTest = getBoolean(querySpecificConfig, CompilerProperties.COMPILER_CBO_TEST_KEY,
+                compilerProperties.getCBOTestMode());
+        boolean forceJoinOrder = getBoolean(querySpecificConfig, CompilerProperties.COMPILER_FORCE_JOIN_ORDER_KEY,
+                compilerProperties.getForceJoinOrderMode());
+        String queryPlanShape = getString(querySpecificConfig, CompilerProperties.COMPILER_QUERY_PLAN_SHAPE_KEY,
+                compilerProperties.getQueryPlanShapeMode());
 
         PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
         physOptConf.setFrameSize(frameSize);
@@ -95,6 +105,11 @@
         physOptConf.setMinMemoryAllocation(minMemoryAllocation);
         physOptConf.setArrayIndexEnabled(arrayIndex);
         physOptConf.setExternalScanBufferSize(externalScanBufferSize);
+        physOptConf.setBatchLookup(batchLookup);
+        physOptConf.setCBOMode(cbo);
+        physOptConf.setCBOTestMode(cboTest);
+        physOptConf.setForceJoinOrderMode(forceJoinOrder);
+        physOptConf.setQueryPlanShapeMode(queryPlanShape);
         return physOptConf;
     }
 
@@ -117,6 +132,14 @@
                 sourceLoc);
     }
 
+    public static int getGroupByNumFrames(CompilerProperties compilerProperties,
+            Map<String, Object> querySpecificConfig, SourceLocation sourceLoc) throws AlgebricksException {
+        return getFrameLimit(CompilerProperties.COMPILER_GROUPMEMORY_KEY,
+                (String) querySpecificConfig.get(CompilerProperties.COMPILER_GROUPMEMORY_KEY),
+                compilerProperties.getGroupMemorySize(), compilerProperties.getFrameSize(),
+                MIN_FRAME_LIMIT_FOR_GROUP_BY, sourceLoc);
+    }
+
     public static int getTextSearchNumFrames(CompilerProperties compilerProperties,
             Map<String, Object> querySpecificConfig, SourceLocation sourceLoc) throws AlgebricksException {
         return getFrameLimit(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
@@ -164,4 +187,12 @@
         }
         return defaultValue;
     }
+
+    private static String getString(Map<String, Object> queryConfig, String queryConfigKey, String defaultValue) {
+        String valueInQuery = (String) queryConfig.get(queryConfigKey);
+        if (valueInQuery != null) {
+            return valueInQuery;
+        }
+        return defaultValue;
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
index 249505b..58d5f9b 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
@@ -72,6 +72,10 @@
         this.partition = partition;
     }
 
+    public IResource getResource() {
+        return resource;
+    }
+
     @Override
     public IIndex createInstance(INCServiceContext ncServiceCtx) throws HyracksDataException {
         return resource.createInstance(ncServiceCtx);
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 fc234ef..2235005 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
@@ -265,6 +265,7 @@
     S3_REGION_NOT_SUPPORTED(1170),
     COMPILATION_SET_OPERATION_ERROR(1171),
     INVALID_TIMEZONE(1172),
+    INVALID_PARAM_VALUE_ALLOWED_VALUE(1173),
 
     // Feed errors
     DATAFLOW_ILLEGAL_STATE(3001),
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/NoOpWarningCollector.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/NoOpWarningCollector.java
index a036b7e..6ac805b 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/NoOpWarningCollector.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/NoOpWarningCollector.java
@@ -20,27 +20,10 @@
 package org.apache.asterix.common.exceptions;
 
 import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.Warning;
 
-public final class NoOpWarningCollector implements IWarningCollector {
-
-    public static final IWarningCollector INSTANCE = new NoOpWarningCollector();
+public final class NoOpWarningCollector {
+    public static final IWarningCollector INSTANCE = org.apache.hyracks.api.exceptions.NoOpWarningCollector.INSTANCE;
 
     private NoOpWarningCollector() {
     }
-
-    @Override
-    public void warn(Warning warning) {
-        // no-op
-    }
-
-    @Override
-    public boolean shouldWarn() {
-        return false;
-    }
-
-    @Override
-    public long getTotalWarningsCount() {
-        return 0;
-    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
index 93fe92d..acbce6d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
@@ -24,13 +24,16 @@
 import java.net.URI;
 import java.security.MessageDigest;
 import java.util.List;
+import java.util.function.Function;
 
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.external.ipc.ExternalFunctionResultRouter;
+import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.ipc.impl.IPCSystem;
 
 public interface ILibraryManager {
@@ -59,9 +62,13 @@
 
     IPCSystem getIPCI();
 
+    NodeControllerService getNcs();
+
     MessageDigest download(FileReference targetFile, String authToken, URI libLocation) throws HyracksException;
 
     void unzip(FileReference sourceFile, FileReference outputDir) throws IOException;
 
     void writeAndForce(FileReference outputFile, InputStream dataStream, byte[] copyBuf) throws IOException;
+
+    void setUploadClient(Function<ILibraryManager, CloseableHttpClient> f);
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
index 28eb553..d830868 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
@@ -52,6 +52,12 @@
     void refreshDatasetBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
             String datasetName) throws AlgebricksException;
 
+    void analyzeDatasetBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
+            String datasetName) throws AlgebricksException;
+
+    void analyzeDatasetDropBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
+            String datasetName) throws AlgebricksException;
+
     void compactBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName, String datasetName)
             throws AlgebricksException;
 
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java
index 232c8dd..c1c6f18 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java
@@ -20,11 +20,14 @@
 package org.apache.asterix.common.utils;
 
 import java.util.EnumSet;
+import java.util.List;
 
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 
 public class JobUtils {
 
@@ -53,4 +56,21 @@
         }
         return jobId;
     }
+
+    public static Pair<JobId, List<IOperatorStats>> runJob(IHyracksClientConnection hcc, JobSpecification spec,
+            EnumSet<JobFlag> jobFlags, boolean waitForCompletion, List<String> statOperatorNames) throws Exception {
+        spec.setMaxReattempts(0);
+        final JobId jobId = hcc.startJob(spec, jobFlags);
+        List<IOperatorStats> opStats = null;
+        if (waitForCompletion) {
+            String nameBefore = Thread.currentThread().getName();
+            try {
+                Thread.currentThread().setName(nameBefore + " : WaitForCompletionForJobId: " + jobId);
+                opStats = hcc.waitForCompletion(jobId, statOperatorNames);
+            } finally {
+                Thread.currentThread().setName(nameBefore);
+            }
+        }
+        return new Pair<>(jobId, opStats);
+    }
 }
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 b98dc43..7455a3b 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -267,6 +267,7 @@
 1170 = Provided S3 region is not supported: '%1$s'
 1171 = Unable to process %1$s clause. %2$s
 1172 = Provided timezone is invalid: '%1$s'
+1173 = Invalid value for parameter '%1$s', allowed value(s): %2$s
 
 # Feed Errors
 3001 = Illegal state.
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss
index 94311f8..68c12f4 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss
@@ -46,6 +46,7 @@
 .options {
     display: flex;
     flex-flow: row;
+    overflow: auto;
 }
 
 .codemirror-container {
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
index f885edb..cb46a6a 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
@@ -538,7 +538,7 @@
         this.queryString = '';
         this.selected = 'Default';
       } else {
-        this.queryString = 'USE ' + this.selected + '; \n';
+        this.queryString = 'USE ' + this.selected + '; \n' + this.queryString;
       }
       this.editor.getDoc().setValue(this.queryString);
       this.editor.execCommand('goDocEnd')
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html
index a131353..4b6ecc5 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html
@@ -97,7 +97,7 @@
         [links]="edgesArr"
         [nodes]="nodesArr"
         [draggingEnabled]="false"
-        [zoomSpeed]="0.025"
+        [zoomSpeed]="0.015"
         [update$]="update$"
         [layoutSettings]="{
                 orientation: planOrientation,
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss
index 6c40a68..9601675 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss
@@ -36,7 +36,6 @@
 }
 
 .panel {
-  order: 2;
   display: flex;
   flex-flow: column;
   justify-content: stretch;
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
index 0fbf6f6..c5754762 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
@@ -30,7 +30,8 @@
                     <button *ngIf="isCSV" mat-button class='button export' (click)='openJSONExportPicker()' matTooltip="Export CSV file to Computer">EXPORT</button>
                     <button *ngIf="isCSV == false" mat-button class='button export' (click)='openJSONExportPicker()' matTooltip="Export JSON/JSONL file to Computer">EXPORT</button>
                 </span>
-                <mat-paginator *ngIf="this.planVisible === false" [showFirstLastButtons]="true" [length]='metrics.resultCount' [pageSize]='pagedefaults.pageSize' [pageSizeOptions]='pageSizeOptions' (page)='showResults($event, false)'>
+
+                <mat-paginator *ngIf="this.planVisible === false" [showFirstLastButtons]="true" [length]='metrics.resultCount' [pageSize]='pageSize' [pageSizeOptions]='pageSizeOptions' (page)='showResults($event, false)'>
                 </mat-paginator>
             </div>
             <div *ngIf='treeVisible' class='navi-data' class='navi-data'>
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
index 5309991..66a6dd6 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
@@ -60,7 +60,8 @@
     currentRange: any;
     /* see 10 records as initial set */
     pagedefaults: any = { pageIndex: 0, pageSize:10, lenght: 0};
-    pageSizeOptions = [5, 10, 25, 100, 200];
+    pageSize = 10;
+    pageSizeOptions = [5, 10, 25, 100, 200, 300, 400];
     viewMode = 'JSON';
     showGoTop = false;
     showGoBottom = false;
@@ -141,6 +142,9 @@
     }
 
     showResults(range, expanded) {
+        // update pagesize
+        this.pageSize = range.pageSize;
+
         this.currentRange = range;
         this.currentIndex = this.currentRange.pageIndex;
 
diff --git a/asterixdb/asterix-docker/pom.xml b/asterixdb/asterix-docker/pom.xml
deleted file mode 100644
index 6c54337..0000000
--- a/asterixdb/asterix-docker/pom.xml
+++ /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.
- !-->
-
-<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/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>apache-asterixdb</artifactId>
-    <groupId>org.apache.asterix</groupId>
-    <version>0.9.8-SNAPSHOT</version>
-  </parent>
-  <artifactId>asterix-docker</artifactId>
-
-  <properties>
-    <root.dir>${basedir}/..</root.dir>
-  </properties>
-
-  <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>
-
-  <profiles>
-    <profile>
-        <id>docker</id>
-        <build>
-          <plugins>
-            <plugin>
-              <groupId>com.spotify</groupId>
-              <artifactId>docker-maven-plugin</artifactId>
-              <version>0.2.11</version>
-              <configuration>
-                <imageName>asterixdb/demo</imageName>
-                <dockerDirectory>docker</dockerDirectory>
-                <resources>
-                  <resource>
-                    <targetPath>/</targetPath>
-                    <directory>../asterix-server/target/</directory>
-                    <include>asterix-server-${project.version}-binary-assembly.zip</include>
-                  </resource>
-                </resources>
-              </configuration>
-            </plugin>
-          </plugins>
-        </build>
-    </profile>
-  </profiles>
-
-</project>
diff --git a/asterixdb/asterix-external-data/pom.xml b/asterixdb/asterix-external-data/pom.xml
index 95ff315..dea4278 100644
--- a/asterixdb/asterix-external-data/pom.xml
+++ b/asterixdb/asterix-external-data/pom.xml
@@ -518,6 +518,10 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>com.google.cloud.bigdataoss</groupId>
+      <artifactId>gcs-connector</artifactId>
+    </dependency>
     <!-- apply patch for HADOOP-17225 to workaround CVE-2019-10172 -->
     <dependency>
       <groupId>org.codehaus.jackson</groupId>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalLangIPCProto.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalLangIPCProto.java
new file mode 100644
index 0000000..35e5961
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalLangIPCProto.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.ipc.MessageType;
+import org.apache.asterix.external.library.msgpack.MsgPackPointableVisitor;
+import org.apache.asterix.om.pointables.AFlatValuePointable;
+import org.apache.asterix.om.pointables.AListVisitablePointable;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.PointableAllocator;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeTagUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+public interface IExternalLangIPCProto {
+    static void visitValueRef(IAType type, DataOutput out, IValueReference valueReference,
+            PointableAllocator pointableAllocator, MsgPackPointableVisitor pointableVisitor, boolean visitNull)
+            throws IOException {
+        IVisitablePointable pointable;
+        switch (type.getTypeTag()) {
+            case OBJECT:
+                pointable = pointableAllocator.allocateRecordValue(type);
+                pointable.set(valueReference);
+                pointableVisitor.visit((ARecordVisitablePointable) pointable, pointableVisitor.getTypeInfo(type, out));
+                break;
+            case ARRAY:
+            case MULTISET:
+                pointable = pointableAllocator.allocateListValue(type);
+                pointable.set(valueReference);
+                pointableVisitor.visit((AListVisitablePointable) pointable, pointableVisitor.getTypeInfo(type, out));
+                break;
+            case ANY:
+                ATypeTag rtTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+                        .deserialize(valueReference.getByteArray()[valueReference.getStartOffset()]);
+                IAType rtType = TypeTagUtil.getBuiltinTypeByTag(rtTypeTag);
+                visitValueRef(rtType, out, valueReference, pointableAllocator, pointableVisitor, visitNull);
+                break;
+            case MISSING:
+            case NULL:
+                if (!visitNull) {
+                    return;
+                }
+            default:
+                pointable = pointableAllocator.allocateFieldValue(type);
+                pointable.set(valueReference);
+                pointableVisitor.visit((AFlatValuePointable) pointable, pointableVisitor.getTypeInfo(type, out));
+                break;
+        }
+    }
+
+    void start();
+
+    void helo() throws IOException, AsterixException;
+
+    long init(String module, String clazz, String fn) throws IOException, AsterixException;
+
+    ByteBuffer call(long functionId, IAType[] argTypes, IValueReference[] argValues, boolean nullCall)
+            throws IOException, AsterixException;
+
+    ByteBuffer callMulti(long key, ArrayBackedValueStorage args, int numTuples) throws IOException, AsterixException;
+
+    //For future use with interpreter reuse between jobs.
+    void quit() throws HyracksDataException;
+
+    void receiveMsg() throws IOException, AsterixException;
+
+    void sendHeader(long key, int msgLen) throws IOException;
+
+    void sendMsg(ArrayBackedValueStorage content) throws IOException;
+
+    void sendMsg() throws IOException;
+
+    MessageType getResponseType();
+
+    long getRouteId();
+
+    DataOutputStream getSockOut();
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/ILibraryEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/ILibraryEvaluator.java
new file mode 100644
index 0000000..8c6538b
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/ILibraryEvaluator.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.external.api;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.om.functions.IExternalFunctionInfo;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.resources.IDeallocatable;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+public interface ILibraryEvaluator extends IDeallocatable {
+
+    void start() throws IOException, AsterixException;
+
+    long initialize(IExternalFunctionInfo finfo) throws IOException, AsterixException;
+
+    ByteBuffer call(long id, IAType[] argTypes, IValueReference[] valueReferences, boolean nullCall) throws IOException;
+
+    ByteBuffer callMulti(long id, ArrayBackedValueStorage arguments, int numTuples) throws IOException;
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
index f14af53..bbcf9cd 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
@@ -18,7 +18,6 @@
  */
 package org.apache.asterix.external.input.record.reader.aws;
 
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3;
 import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
 
 import java.io.IOException;
@@ -32,7 +31,7 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
 import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.aws.s3.S3Utils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.util.CleanupUtils;
@@ -109,7 +108,7 @@
     }
 
     private boolean shouldRetry(String errorCode, int currentRetry) {
-        return currentRetry < MAX_RETRIES && AwsS3.isRetryableError(errorCode);
+        return currentRetry < MAX_RETRIES && S3Utils.isRetryableError(errorCode);
     }
 
     @Override
@@ -134,7 +133,7 @@
 
     private S3Client buildAwsS3Client(Map<String, String> configuration) throws HyracksDataException {
         try {
-            return ExternalDataUtils.AwsS3.buildAwsS3Client(configuration);
+            return S3Utils.buildAwsS3Client(configuration);
         } catch (CompilationException ex) {
             throw HyracksDataException.create(ex);
         }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
index 89ea39e..a241354 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.external.api.AsterixInputStream;
 import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
 import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.aws.s3.S3Utils;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.IServiceContext;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -53,8 +54,7 @@
         IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
 
         //Get a list of S3 objects
-        List<S3Object> filesOnly =
-                ExternalDataUtils.AwsS3.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
+        List<S3Object> filesOnly = S3Utils.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
         // Distribute work load amongst the partitions
         distributeWorkLoad(filesOnly, getPartitionsCount());
     }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java
index 4e3d1ec..92b7a95 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java
@@ -32,6 +32,8 @@
 import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.aws.s3.S3Constants;
+import org.apache.asterix.external.util.aws.s3.S3Utils;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.IServiceContext;
@@ -61,7 +63,7 @@
         try {
             JobConf conf = createHdfsConf(serviceCtx, configuration);
             int numberOfPartitions = getPartitionConstraint().getLocations().length;
-            ExternalDataUtils.AwsS3.configureAwsS3HdfsJobConf(conf, configuration, numberOfPartitions);
+            S3Utils.configureAwsS3HdfsJobConf(conf, configuration, numberOfPartitions);
             configureHdfsConf(conf, configuration);
         } catch (SdkException | SdkBaseException ex) {
             throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
@@ -107,8 +109,7 @@
             throws CompilationException {
         String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
         IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
-        List<S3Object> filesOnly =
-                ExternalDataUtils.AwsS3.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
+        List<S3Object> filesOnly = S3Utils.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
         StringBuilder builder = new StringBuilder();
 
         if (!filesOnly.isEmpty()) {
@@ -123,7 +124,7 @@
     }
 
     private static void appendFileURI(StringBuilder builder, String container, S3Object file) {
-        builder.append(ExternalDataConstants.AwsS3.HADOOP_S3_PROTOCOL);
+        builder.append(S3Constants.HADOOP_S3_PROTOCOL);
         builder.append("://");
         builder.append(container);
         builder.append('/');
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java
index cdb3834..bbfece2 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.external.input.record.reader.azure.blob;
 
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureBlobClient;
 import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
 
 import java.io.IOException;
@@ -31,7 +32,6 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
 import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.util.LogRedactionUtil;
 
@@ -86,7 +86,7 @@
     private BlobServiceClient buildAzureClient(IApplicationContext appCtx, Map<String, String> configuration)
             throws HyracksDataException {
         try {
-            return ExternalDataUtils.Azure.buildAzureBlobClient(appCtx, configuration);
+            return buildAzureBlobClient(appCtx, configuration);
         } catch (CompilationException ex) {
             throw HyracksDataException.create(ex);
         }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java
index 064b319..55c0521 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.external.input.record.reader.azure.blob;
 
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureBlobClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listBlobItems;
+
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
@@ -57,9 +60,9 @@
         // Ensure the validity of include/exclude
         ExternalDataUtils.validateIncludeExclude(configuration);
         IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
-        BlobServiceClient blobServiceClient = ExternalDataUtils.Azure.buildAzureBlobClient(appCtx, configuration);
-        List<BlobItem> filesOnly = ExternalDataUtils.Azure.listBlobItems(blobServiceClient, configuration,
-                includeExcludeMatcher, warningCollector);
+        BlobServiceClient blobServiceClient = buildAzureBlobClient(appCtx, configuration);
+        List<BlobItem> filesOnly =
+                listBlobItems(blobServiceClient, configuration, includeExcludeMatcher, warningCollector);
 
         // Distribute work load amongst the partitions
         distributeWorkLoad(filesOnly, getPartitionsCount());
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java
index e34d188..7a95222 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.external.input.record.reader.azure.datalake;
 
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureDatalakeClient;
 import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
 
 import java.io.IOException;
@@ -31,7 +32,6 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
 import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.util.LogRedactionUtil;
 
@@ -86,7 +86,7 @@
     private DataLakeServiceClient buildAzureClient(IApplicationContext appCtx, Map<String, String> configuration)
             throws HyracksDataException {
         try {
-            return ExternalDataUtils.Azure.buildAzureDatalakeClient(appCtx, configuration);
+            return buildAzureDatalakeClient(appCtx, configuration);
         } catch (CompilationException ex) {
             throw HyracksDataException.create(ex);
         }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java
index e9f8d4c..929cb6e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.external.input.record.reader.azure.datalake;
 
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureDatalakeClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listDatalakePathItems;
+
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
@@ -57,9 +60,9 @@
         // Ensure the validity of include/exclude
         ExternalDataUtils.validateIncludeExclude(configuration);
         IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
-        DataLakeServiceClient client = ExternalDataUtils.Azure.buildAzureDatalakeClient(appCtx, configuration);
-        List<PathItem> filesOnly = ExternalDataUtils.Azure.listDatalakePathItems(client, configuration,
-                includeExcludeMatcher, warningCollector);
+        DataLakeServiceClient client = buildAzureDatalakeClient(appCtx, configuration);
+        List<PathItem> filesOnly =
+                listDatalakePathItems(client, configuration, includeExcludeMatcher, warningCollector);
 
         // Distribute work load amongst the partitions
         distributeWorkLoad(filesOnly, getPartitionsCount());
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java
index c2251df..e08013c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java
@@ -18,6 +18,11 @@
  */
 package org.apache.asterix.external.input.record.reader.azure.parquet;
 
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_BLOB_PROTOCOL;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureBlobClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.configureAzureHdfsJobConf;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listBlobItems;
+
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -47,7 +52,7 @@
     public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
             IWarningCollector warningCollector) throws AlgebricksException, HyracksDataException {
         IApplicationContext appCtx = (IApplicationContext) serviceCtx.getApplicationContext();
-        BlobServiceClient blobServiceClient = ExternalDataUtils.Azure.buildAzureBlobClient(appCtx, configuration);
+        BlobServiceClient blobServiceClient = buildAzureBlobClient(appCtx, configuration);
         //Get endpoint
         String endPoint = extractEndPoint(blobServiceClient.getAccountUrl());
         //Get path
@@ -57,7 +62,7 @@
 
         //Configure Hadoop Azure input splits
         JobConf conf = createHdfsConf(serviceCtx, configuration);
-        ExternalDataUtils.Azure.configureAzureHdfsJobConf(conf, configuration, endPoint);
+        configureAzureHdfsJobConf(conf, configuration, endPoint);
         configureHdfsConf(conf, configuration);
     }
 
@@ -94,8 +99,8 @@
     private static String buildPathURIs(Map<String, String> configuration, IWarningCollector warningCollector,
             BlobServiceClient blobServiceClient, String endPoint) throws CompilationException {
         IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
-        List<BlobItem> filesOnly = ExternalDataUtils.Azure.listBlobItems(blobServiceClient, configuration,
-                includeExcludeMatcher, warningCollector);
+        List<BlobItem> filesOnly =
+                listBlobItems(blobServiceClient, configuration, includeExcludeMatcher, warningCollector);
 
         StringBuilder builder = new StringBuilder();
         String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
@@ -118,7 +123,7 @@
     }
 
     private static void appendFileURI(StringBuilder builder, String container, String endPoint, BlobItem file) {
-        builder.append(ExternalDataConstants.Azure.HADOOP_AZURE_BLOB_PROTOCOL);
+        builder.append(HADOOP_AZURE_BLOB_PROTOCOL);
         builder.append("://");
         builder.append(container);
         builder.append('@');
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java
index db87868..c98fc8b 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java
@@ -18,6 +18,11 @@
  */
 package org.apache.asterix.external.input.record.reader.azure.parquet;
 
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_DATALAKE_PROTOCOL;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureDatalakeClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.configureAzureHdfsJobConf;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listDatalakePathItems;
+
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -47,8 +52,7 @@
     public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
             IWarningCollector warningCollector) throws AlgebricksException, HyracksDataException {
         IApplicationContext appCtx = (IApplicationContext) serviceCtx.getApplicationContext();
-        DataLakeServiceClient dataLakeServiceClient =
-                ExternalDataUtils.Azure.buildAzureDatalakeClient(appCtx, configuration);
+        DataLakeServiceClient dataLakeServiceClient = buildAzureDatalakeClient(appCtx, configuration);
 
         //Get endpoint
         String endPoint = extractEndPoint(dataLakeServiceClient.getAccountUrl());
@@ -61,7 +65,7 @@
 
         //Configure Hadoop Azure input splits
         JobConf conf = createHdfsConf(serviceCtx, configuration);
-        ExternalDataUtils.Azure.configureAzureHdfsJobConf(conf, configuration, endPoint);
+        configureAzureHdfsJobConf(conf, configuration, endPoint);
         configureHdfsConf(conf, configuration);
     }
 
@@ -98,8 +102,8 @@
     private static String buildPathURIs(Map<String, String> configuration, IWarningCollector warningCollector,
             DataLakeServiceClient dataLakeServiceClient, String endPoint) throws CompilationException {
         IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
-        List<PathItem> filesOnly = ExternalDataUtils.Azure.listDatalakePathItems(dataLakeServiceClient, configuration,
-                includeExcludeMatcher, warningCollector);
+        List<PathItem> filesOnly =
+                listDatalakePathItems(dataLakeServiceClient, configuration, includeExcludeMatcher, warningCollector);
 
         StringBuilder builder = new StringBuilder();
         String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
@@ -122,7 +126,7 @@
     }
 
     private static void appendFileURI(StringBuilder builder, String container, String endPoint, PathItem file) {
-        builder.append(ExternalDataConstants.Azure.HADOOP_AZURE_DATALAKE_PROTOCOL);
+        builder.append(HADOOP_AZURE_DATALAKE_PROTOCOL);
         builder.append("://");
         builder.append(container);
         builder.append('@');
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
index 652fa3e..007e8be 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
@@ -32,7 +32,7 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
 import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.google.gcs.GCSUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.util.CleanupUtils;
@@ -134,7 +134,7 @@
 
     private Storage buildClient(Map<String, String> configuration) throws HyracksDataException {
         try {
-            return ExternalDataUtils.GCS.buildClient(configuration);
+            return GCSUtils.buildClient(configuration);
         } catch (CompilationException ex) {
             throw HyracksDataException.create(ex);
         }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java
index 0e7ea90..278c1ad 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java
@@ -19,33 +19,23 @@
 package org.apache.asterix.external.input.record.reader.gcs;
 
 import static org.apache.asterix.external.util.ExternalDataUtils.getIncludeExcludeMatchers;
-import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
 
-import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
 import java.util.PriorityQueue;
-import java.util.function.BiPredicate;
-import java.util.regex.Matcher;
 
-import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.api.AsterixInputStream;
 import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
-import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.google.gcs.GCSUtils;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.IServiceContext;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.Warning;
 
-import com.google.api.gax.paging.Page;
-import com.google.cloud.BaseServiceException;
 import com.google.cloud.storage.Blob;
-import com.google.cloud.storage.Storage;
 
 public class GCSInputStreamFactory extends AbstractExternalInputStreamFactory {
 
@@ -63,57 +53,16 @@
 
         // Ensure the validity of include/exclude
         ExternalDataUtils.validateIncludeExclude(configuration);
-
-        // Prepare to retrieve the objects
-        List<Blob> filesOnly = new ArrayList<>();
-        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-        Storage gcs = ExternalDataUtils.GCS.buildClient(configuration);
-        Storage.BlobListOption options = Storage.BlobListOption.prefix(ExternalDataUtils.getPrefix(configuration));
-        Page<Blob> items;
-
-        try {
-            items = gcs.list(container, options);
-        } catch (BaseServiceException ex) {
-            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-        }
-
-        // Collect the paths to files only
         IncludeExcludeMatcher includeExcludeMatcher = getIncludeExcludeMatchers(configuration);
-        collectAndFilterFiles(items, includeExcludeMatcher.getPredicate(), includeExcludeMatcher.getMatchersList(),
-                filesOnly);
 
-        // Warn if no files are returned
-        if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
-            Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
-            warningCollector.warn(warning);
-        }
+        // get the items
+        List<Blob> filesOnly = GCSUtils.listItems(configuration, includeExcludeMatcher, warningCollector);
 
         // Distribute work load amongst the partitions
         distributeWorkLoad(filesOnly, getPartitionsCount());
     }
 
     /**
-     * AWS S3 returns all the objects as paths, not differentiating between folder and files. The path is considered
-     * a file if it does not end up with a "/" which is the separator in a folder structure.
-     *
-     * @param items List of returned objects
-     */
-    private void collectAndFilterFiles(Page<Blob> items, BiPredicate<List<Matcher>, String> predicate,
-            List<Matcher> matchers, List<Blob> filesOnly) {
-        for (Blob item : items.iterateAll()) {
-            // skip folders
-            if (item.getName().endsWith("/")) {
-                continue;
-            }
-
-            // No filter, add file
-            if (predicate.test(matchers, item.getName())) {
-                filesOnly.add(item);
-            }
-        }
-    }
-
-    /**
      * To efficiently utilize the parallelism, work load will be distributed amongst the partitions based on the file
      * size.
      *
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/parquet/GCSParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/parquet/GCSParquetReaderFactory.java
new file mode 100644
index 0000000..2887415
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/parquet/GCSParquetReaderFactory.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.asterix.external.input.record.reader.gcs.parquet;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.external.input.HDFSDataSourceFactory;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.google.gcs.GCSConstants;
+import org.apache.asterix.external.util.google.gcs.GCSUtils;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+
+import com.google.cloud.storage.Blob;
+
+public class GCSParquetReaderFactory extends HDFSDataSourceFactory {
+    private static final long serialVersionUID = -6140824803254158253L;
+    private static final List<String> recordReaderNames =
+            Collections.singletonList(ExternalDataConstants.KEY_ADAPTER_NAME_GCS);
+
+    @Override
+    public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
+            IWarningCollector warningCollector) throws AlgebricksException, HyracksDataException {
+        // get path
+        String path = buildPathURIs(configuration, warningCollector);
+
+        // put GCS configurations to AsterixDB's Hadoop configuration
+        putGCSConfToHadoopConf(configuration, path);
+
+        // configure hadoop input splits
+        JobConf conf = createHdfsConf(serviceCtx, configuration);
+        int numberOfPartitions = getPartitionConstraint().getLocations().length;
+        GCSUtils.configureHdfsJobConf(conf, configuration, numberOfPartitions);
+        configureHdfsConf(conf, configuration);
+    }
+
+    @Override
+    public List<String> getRecordReaderNames() {
+        return recordReaderNames;
+    }
+
+    @Override
+    public Set<String> getReaderSupportedFormats() {
+        return Collections.singleton(ExternalDataConstants.FORMAT_PARQUET);
+    }
+
+    /**
+     * Prepare Hadoop configurations to read parquet files
+     *
+     * @param path Comma-delimited paths
+     */
+    private static void putGCSConfToHadoopConf(Map<String, String> configuration, String path) {
+        configuration.put(ExternalDataConstants.KEY_PATH, path);
+        configuration.put(ExternalDataConstants.KEY_INPUT_FORMAT, ExternalDataConstants.INPUT_FORMAT_PARQUET);
+        configuration.put(ExternalDataConstants.KEY_PARSER, ExternalDataConstants.FORMAT_NOOP);
+    }
+
+    /**
+     * Build Google Cloud Storage path-style for the requested files
+     *
+     * @param configuration    properties
+     * @param warningCollector warning collector
+     * @return Comma-delimited paths (e.g., "gs://bucket/file1.parquet,gs://bucket/file2.parquet")
+     * @throws CompilationException Compilation exception
+     */
+    private static String buildPathURIs(Map<String, String> configuration, IWarningCollector warningCollector)
+            throws CompilationException {
+        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+        // Ensure the validity of include/exclude
+        ExternalDataUtils.validateIncludeExclude(configuration);
+        IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
+
+        // get the items
+        List<Blob> filesOnly = GCSUtils.listItems(configuration, includeExcludeMatcher, warningCollector);
+        StringBuilder builder = new StringBuilder();
+
+        if (!filesOnly.isEmpty()) {
+            appendFileURI(builder, container, filesOnly.get(0));
+            for (int i = 1; i < filesOnly.size(); i++) {
+                builder.append(',');
+                appendFileURI(builder, container, filesOnly.get(i));
+            }
+        }
+
+        return builder.toString();
+    }
+
+    private static void appendFileURI(StringBuilder builder, String container, Blob file) {
+        builder.append(GCSConstants.HADOOP_GCS_PROTOCOL);
+        builder.append("://");
+        builder.append(container);
+        builder.append('/');
+        builder.append(file.getName());
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/AbstractPythonIPCProto.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/AbstractPythonIPCProto.java
new file mode 100644
index 0000000..00d1dcc
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/AbstractPythonIPCProto.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.external.ipc;
+
+import static org.apache.hyracks.ipc.impl.Message.HEADER_SIZE;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.api.IExternalLangIPCProto;
+import org.apache.asterix.external.library.msgpack.MsgPackPointableVisitor;
+import org.apache.asterix.om.pointables.PointableAllocator;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.ipc.impl.Message;
+import org.msgpack.core.MessagePack;
+import org.msgpack.core.MessageUnpacker;
+import org.msgpack.core.buffer.ArrayBufferInput;
+
+public abstract class AbstractPythonIPCProto {
+    public static final int HEADER_SIZE_LEN_INCLUSIVE = 21;
+    protected final PythonMessageBuilder messageBuilder;
+    protected final DataOutputStream sockOut;
+    protected final ArrayBufferInput unpackerInput;
+    protected final MessageUnpacker unpacker;
+    protected final ArrayBackedValueStorage argsStorage;
+    protected final PointableAllocator pointableAllocator;
+    protected final MsgPackPointableVisitor pointableVisitor;
+    private final ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_SIZE_LEN_INCLUSIVE);
+    protected ByteBuffer recvBuffer = ByteBuffer.allocate(32768);
+    protected long routeId;
+    protected Pair<ByteBuffer, Exception> bufferBox;
+    protected long maxFunctionId;
+
+    public AbstractPythonIPCProto(OutputStream sockOut) {
+        messageBuilder = new PythonMessageBuilder();
+        this.sockOut = new DataOutputStream(sockOut);
+        this.maxFunctionId = 0L;
+        unpackerInput = new ArrayBufferInput(new byte[0]);
+        unpacker = MessagePack.newDefaultUnpacker(unpackerInput);
+        this.argsStorage = new ArrayBackedValueStorage();
+        this.pointableAllocator = new PointableAllocator();
+        this.pointableVisitor = new MsgPackPointableVisitor();
+    }
+
+    public void helo() throws IOException, AsterixException {
+        recvBuffer.clear();
+        recvBuffer.position(0);
+        recvBuffer.limit(0);
+        messageBuilder.reset();
+        messageBuilder.hello();
+        sendHeader(routeId, messageBuilder.getLength());
+        sendMsg();
+        receiveMsg();
+        if (getResponseType() != MessageType.HELO) {
+            throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+                    "Expected HELO, recieved " + getResponseType().name());
+        }
+    }
+
+    public long init(String module, String clazz, String fn) throws IOException, AsterixException {
+        long functionId = maxFunctionId++;
+        recvBuffer.clear();
+        recvBuffer.position(0);
+        recvBuffer.limit(0);
+        messageBuilder.reset();
+        messageBuilder.init(module, clazz, fn);
+        sendHeader(functionId, messageBuilder.getLength());
+        sendMsg();
+        receiveMsg();
+        if (getResponseType() != MessageType.INIT_RSP) {
+            throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+                    "Expected INIT_RSP, recieved " + getResponseType().name());
+        }
+        return functionId;
+    }
+
+    public ByteBuffer call(long functionId, IAType[] argTypes, IValueReference[] argValues, boolean nullCall)
+            throws IOException, AsterixException {
+        recvBuffer.clear();
+        recvBuffer.position(0);
+        recvBuffer.limit(0);
+        messageBuilder.reset();
+        argsStorage.reset();
+        for (int i = 0; i < argTypes.length; i++) {
+            IExternalLangIPCProto.visitValueRef(argTypes[i], argsStorage.getDataOutput(), argValues[i],
+                    pointableAllocator, pointableVisitor, nullCall);
+        }
+        int len = argsStorage.getLength() + 5;
+        sendHeader(functionId, len);
+        messageBuilder.call(argValues.length, len);
+        sendMsg(argsStorage);
+        receiveMsg();
+        if (getResponseType() != MessageType.CALL_RSP) {
+            throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+                    "Expected CALL_RSP, recieved " + getResponseType().name());
+        }
+        return recvBuffer;
+    }
+
+    public ByteBuffer callMulti(long key, ArrayBackedValueStorage args, int numTuples)
+            throws IOException, AsterixException {
+        recvBuffer.clear();
+        recvBuffer.position(0);
+        recvBuffer.limit(0);
+        messageBuilder.reset();
+        int len = args.getLength() + 4;
+        sendHeader(key, len);
+        messageBuilder.callMulti(0, numTuples);
+        sendMsg(args);
+        receiveMsg();
+        if (getResponseType() != MessageType.CALL_RSP) {
+            throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+                    "Expected CALL_RSP, recieved " + getResponseType().name());
+        }
+        return recvBuffer;
+    }
+
+    public void quit() throws HyracksDataException {
+        messageBuilder.quit();
+    }
+
+    public abstract void receiveMsg() throws IOException, AsterixException;
+
+    public void sendHeader(long key, int msgLen) throws IOException {
+        headerBuffer.clear();
+        headerBuffer.position(0);
+        headerBuffer.putInt(HEADER_SIZE + Integer.BYTES + msgLen);
+        headerBuffer.putLong(key);
+        headerBuffer.putLong(routeId);
+        headerBuffer.put(Message.NORMAL);
+        sockOut.write(headerBuffer.array(), 0, HEADER_SIZE + Integer.BYTES);
+        sockOut.flush();
+    }
+
+    public void sendMsg(ArrayBackedValueStorage content) throws IOException {
+        sockOut.write(messageBuilder.getBuf().array(), messageBuilder.getBuf().arrayOffset(),
+                messageBuilder.getBuf().position());
+        sockOut.write(content.getByteArray(), content.getStartOffset(), content.getLength());
+        sockOut.flush();
+    }
+
+    public void sendMsg() throws IOException {
+        sockOut.write(messageBuilder.getBuf().array(), messageBuilder.getBuf().arrayOffset(),
+                messageBuilder.getBuf().position());
+        sockOut.flush();
+    }
+
+    public MessageType getResponseType() {
+        return messageBuilder.type;
+    }
+
+    public long getRouteId() {
+        return routeId;
+    }
+
+    public DataOutputStream getSockOut() {
+        return sockOut;
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonDomainSocketProto.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonDomainSocketProto.java
new file mode 100644
index 0000000..89f240a
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonDomainSocketProto.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.external.ipc;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.SocketChannel;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.api.IExternalLangIPCProto;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.msgpack.core.MessagePack;
+
+public class PythonDomainSocketProto extends AbstractPythonIPCProto implements IExternalLangIPCProto {
+    private final String wd;
+    SocketChannel chan;
+    private ByteBuffer headerBuffer;
+    private ProcessHandle pid;
+    public static final int HYR_HEADER_SIZE = 21; // 4 (sz) + 8 (mid) + 8 (rmid) + 1 (flags)
+    public static final int HYR_HEADER_SIZE_NOSZ = 17; // 8 + 8 + 1
+
+    public PythonDomainSocketProto(OutputStream sockOut, SocketChannel chan, String wd) {
+        super(sockOut);
+        this.chan = chan;
+        this.wd = wd;
+        headerBuffer = ByteBuffer.allocate(HYR_HEADER_SIZE);
+    }
+
+    @Override
+    public void start() {
+    }
+
+    @Override
+    public void helo() throws IOException, AsterixException {
+        recvBuffer.clear();
+        recvBuffer.position(0);
+        recvBuffer.limit(0);
+        messageBuilder.reset();
+        messageBuilder.helloDS(wd);
+        sendHeader(routeId, messageBuilder.getLength());
+        sendMsg(true);
+        receiveMsg(true);
+        byte pidType = recvBuffer.get();
+        if (pidType != MessagePack.Code.UINT32 && pidType != MessagePack.Code.UINT16) {
+            throw AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION,
+                    "Returned pid type is incorrect: " + pidType);
+        }
+        switch (pidType) {
+            case MessagePack.Code.UINT32:
+                pid = ProcessHandle.of(recvBuffer.getInt()).get();
+                break;
+            case MessagePack.Code.UINT16:
+                pid = ProcessHandle.of(recvBuffer.getShort()).get();
+                break;
+            case MessagePack.Code.UINT8:
+                pid = ProcessHandle.of(recvBuffer.get()).get();
+                break;
+            default:
+                throw AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION,
+                        "Returned pid type is incorrect: " + pidType);
+        }
+        if (getResponseType() != MessageType.HELO) {
+            throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+                    "Expected HELO, recieved " + getResponseType().name());
+        }
+    }
+
+    @Override
+    public void sendMsg() throws IOException {
+        sendMsg(false);
+    }
+
+    @Override
+    public void sendMsg(ArrayBackedValueStorage args) throws IOException {
+        sendMsg(false, args);
+    }
+
+    public void sendMsg(boolean sendIfDead) throws IOException {
+        if (!sendIfDead && (pid == null || !pid.isAlive())) {
+            return;
+        }
+        super.sendMsg();
+    }
+
+    public void sendMsg(boolean sendIfDead, ArrayBackedValueStorage args) throws IOException {
+        if (!sendIfDead && (pid == null || !pid.isAlive())) {
+            return;
+        }
+        super.sendMsg(args);
+    }
+
+    @Override
+    public void receiveMsg() throws IOException, AsterixException {
+        receiveMsg(false);
+    }
+
+    public void receiveMsg(boolean sendIfDead) throws IOException, AsterixException {
+        if (!sendIfDead && (pid == null || !pid.isAlive())) {
+            throw new AsterixException("Python process exited unexpectedly");
+        }
+        readFully(headerBuffer.capacity(), headerBuffer);
+        if (headerBuffer.remaining() < Integer.BYTES) {
+            recvBuffer.limit(0);
+            throw new AsterixException("Python process exited unexpectedly");
+        }
+        int msgSz = headerBuffer.getInt() - HYR_HEADER_SIZE_NOSZ;
+        if (recvBuffer.capacity() < msgSz) {
+            recvBuffer = ByteBuffer.allocate(((msgSz / 32768) + 1) * 32768);
+        }
+        readFully(msgSz, recvBuffer);
+        messageBuilder.readHead(recvBuffer);
+        if (messageBuilder.type == MessageType.ERROR) {
+            unpackerInput.reset(recvBuffer.array(), recvBuffer.position() + recvBuffer.arrayOffset(),
+                    recvBuffer.remaining());
+            unpacker.reset(unpackerInput);
+            throw new AsterixException(unpacker.unpackString().replace('\0', ' '));
+        }
+    }
+
+    private void readFully(int msgSz, ByteBuffer buf) throws IOException, AsterixException {
+        buf.limit(msgSz);
+        buf.clear();
+        int read;
+        int size = msgSz;
+        while (size > 0) {
+            read = chan.read(buf);
+            if (read < 0) {
+                throw new AsterixException("Socket closed");
+            }
+            size -= read;
+        }
+        buf.flip();
+    }
+
+    @Override
+    public void quit() throws HyracksDataException {
+        messageBuilder.quit();
+    }
+
+    public ProcessHandle getPid() {
+        return pid;
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonIPCProto.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonIPCProto.java
deleted file mode 100644
index c803517..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonIPCProto.java
+++ /dev/null
@@ -1,290 +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.ipc;
-
-import static org.apache.hyracks.ipc.impl.Message.HEADER_SIZE;
-
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.external.library.msgpack.MsgPackPointableVisitor;
-import org.apache.asterix.om.pointables.AFlatValuePointable;
-import org.apache.asterix.om.pointables.AListVisitablePointable;
-import org.apache.asterix.om.pointables.ARecordVisitablePointable;
-import org.apache.asterix.om.pointables.PointableAllocator;
-import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.types.TypeTagUtil;
-import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IValueReference;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.ipc.impl.Message;
-import org.msgpack.core.MessagePack;
-import org.msgpack.core.MessageUnpacker;
-import org.msgpack.core.buffer.ArrayBufferInput;
-
-public class PythonIPCProto {
-
-    private final PythonMessageBuilder messageBuilder;
-    private final DataOutputStream sockOut;
-    private final ByteBuffer headerBuffer = ByteBuffer.allocate(21);
-    private ByteBuffer recvBuffer = ByteBuffer.allocate(32768);
-    private final ExternalFunctionResultRouter router;
-    private long routeId;
-    private Pair<ByteBuffer, Exception> bufferBox;
-    private final Process pythonProc;
-    private long maxFunctionId;
-    private final ArrayBufferInput unpackerInput;
-    private final MessageUnpacker unpacker;
-    private final ArrayBackedValueStorage argsStorage;
-    private final PointableAllocator pointableAllocator;
-    private final MsgPackPointableVisitor pointableVisitor;
-
-    public PythonIPCProto(OutputStream sockOut, ExternalFunctionResultRouter router, Process pythonProc) {
-        this.sockOut = new DataOutputStream(sockOut);
-        messageBuilder = new PythonMessageBuilder();
-        this.router = router;
-        this.pythonProc = pythonProc;
-        this.maxFunctionId = 0L;
-        unpackerInput = new ArrayBufferInput(new byte[0]);
-        unpacker = MessagePack.newDefaultUnpacker(unpackerInput);
-        this.argsStorage = new ArrayBackedValueStorage();
-        this.pointableAllocator = new PointableAllocator();
-        this.pointableVisitor = new MsgPackPointableVisitor();
-    }
-
-    public void start() {
-        Pair<Long, Pair<ByteBuffer, Exception>> keyAndBufferBox = router.insertRoute(recvBuffer);
-        this.routeId = keyAndBufferBox.getFirst();
-        this.bufferBox = keyAndBufferBox.getSecond();
-    }
-
-    public void helo() throws IOException, AsterixException {
-        recvBuffer.clear();
-        recvBuffer.position(0);
-        recvBuffer.limit(0);
-        messageBuilder.reset();
-        messageBuilder.hello();
-        sendHeader(routeId, messageBuilder.getLength());
-        sendMsg();
-        receiveMsg();
-        if (getResponseType() != MessageType.HELO) {
-            throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
-                    "Expected HELO, recieved " + getResponseType().name());
-        }
-    }
-
-    public long init(String module, String clazz, String fn) throws IOException, AsterixException {
-        long functionId = maxFunctionId++;
-        recvBuffer.clear();
-        recvBuffer.position(0);
-        recvBuffer.limit(0);
-        messageBuilder.reset();
-        messageBuilder.init(module, clazz, fn);
-        sendHeader(functionId, messageBuilder.getLength());
-        sendMsg();
-        receiveMsg();
-        if (getResponseType() != MessageType.INIT_RSP) {
-            throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
-                    "Expected INIT_RSP, recieved " + getResponseType().name());
-        }
-        return functionId;
-    }
-
-    public ByteBuffer call(long functionId, IAType[] argTypes, IValueReference[] argValues, boolean nullCall)
-            throws IOException, AsterixException {
-        recvBuffer.clear();
-        recvBuffer.position(0);
-        recvBuffer.limit(0);
-        messageBuilder.reset();
-        argsStorage.reset();
-        for (int i = 0; i < argTypes.length; i++) {
-            visitValueRef(argTypes[i], argsStorage.getDataOutput(), argValues[i], pointableAllocator, pointableVisitor,
-                    nullCall);
-        }
-        int len = argsStorage.getLength() + 5;
-        sendHeader(functionId, len);
-        messageBuilder.call(argValues.length, len);
-        sendMsg(argsStorage);
-        receiveMsg();
-        if (getResponseType() != MessageType.CALL_RSP) {
-            throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
-                    "Expected CALL_RSP, recieved " + getResponseType().name());
-        }
-        return recvBuffer;
-    }
-
-    public ByteBuffer callMulti(long key, ArrayBackedValueStorage args, int numTuples)
-            throws IOException, AsterixException {
-        recvBuffer.clear();
-        recvBuffer.position(0);
-        recvBuffer.limit(0);
-        messageBuilder.reset();
-        int len = args.getLength() + 4;
-        sendHeader(key, len);
-        messageBuilder.callMulti(0, numTuples);
-        sendMsg(args);
-        receiveMsg();
-        if (getResponseType() != MessageType.CALL_RSP) {
-            throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
-                    "Expected CALL_RSP, recieved " + getResponseType().name());
-        }
-        return recvBuffer;
-    }
-
-    //For future use with interpreter reuse between jobs.
-    public void quit() throws HyracksDataException {
-        messageBuilder.quit();
-        router.removeRoute(routeId);
-    }
-
-    public void receiveMsg() throws IOException, AsterixException {
-        Exception except;
-        try {
-            synchronized (bufferBox) {
-                while ((bufferBox.getFirst().limit() == 0 || bufferBox.getSecond() != null) && pythonProc.isAlive()) {
-                    bufferBox.wait(100);
-                }
-            }
-            except = router.getAndRemoveException(routeId);
-            if (!pythonProc.isAlive()) {
-                except = new IOException("Python process exited with code: " + pythonProc.exitValue());
-            }
-        } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            throw new AsterixException(ErrorCode.EXTERNAL_UDF_EXCEPTION, e);
-        }
-        if (except != null) {
-            throw new AsterixException(except);
-        }
-        if (bufferBox.getFirst() != recvBuffer) {
-            recvBuffer = bufferBox.getFirst();
-        }
-        messageBuilder.readHead(recvBuffer);
-        if (messageBuilder.type == MessageType.ERROR) {
-            unpackerInput.reset(recvBuffer.array(), recvBuffer.position() + recvBuffer.arrayOffset(),
-                    recvBuffer.remaining());
-            unpacker.reset(unpackerInput);
-            throw new AsterixException(unpacker.unpackString());
-        }
-    }
-
-    public void sendHeader(long key, int msgLen) throws IOException {
-        headerBuffer.clear();
-        headerBuffer.position(0);
-        headerBuffer.putInt(HEADER_SIZE + Integer.BYTES + msgLen);
-        headerBuffer.putLong(key);
-        headerBuffer.putLong(routeId);
-        headerBuffer.put(Message.NORMAL);
-        sockOut.write(headerBuffer.array(), 0, HEADER_SIZE + Integer.BYTES);
-        sockOut.flush();
-    }
-
-    public void sendMsg(ArrayBackedValueStorage content) throws IOException {
-        sockOut.write(messageBuilder.getBuf().array(), messageBuilder.getBuf().arrayOffset(),
-                messageBuilder.getBuf().position());
-        sockOut.write(content.getByteArray(), content.getStartOffset(), content.getLength());
-        sockOut.flush();
-    }
-
-    public void sendMsg() throws IOException {
-        sockOut.write(messageBuilder.getBuf().array(), messageBuilder.getBuf().arrayOffset(),
-                messageBuilder.getBuf().position());
-        sockOut.flush();
-    }
-
-    public MessageType getResponseType() {
-        return messageBuilder.type;
-    }
-
-    public long getRouteId() {
-        return routeId;
-    }
-
-    public DataOutputStream getSockOut() {
-        return sockOut;
-    }
-
-    public static void visitValueRef(IAType type, DataOutput out, IValueReference valueReference,
-            PointableAllocator pointableAllocator, MsgPackPointableVisitor pointableVisitor, boolean visitNull)
-            throws IOException {
-        IVisitablePointable pointable;
-        switch (type.getTypeTag()) {
-            case OBJECT:
-                pointable = pointableAllocator.allocateRecordValue(type);
-                pointable.set(valueReference);
-                pointableVisitor.visit((ARecordVisitablePointable) pointable, pointableVisitor.getTypeInfo(type, out));
-                break;
-            case ARRAY:
-            case MULTISET:
-                pointable = pointableAllocator.allocateListValue(type);
-                pointable.set(valueReference);
-                pointableVisitor.visit((AListVisitablePointable) pointable, pointableVisitor.getTypeInfo(type, out));
-                break;
-            case ANY:
-                ATypeTag rtTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER
-                        .deserialize(valueReference.getByteArray()[valueReference.getStartOffset()]);
-                IAType rtType = TypeTagUtil.getBuiltinTypeByTag(rtTypeTag);
-                switch (rtTypeTag) {
-                    case OBJECT:
-                        pointable = pointableAllocator.allocateRecordValue(rtType);
-                        pointable.set(valueReference);
-                        pointableVisitor.visit((ARecordVisitablePointable) pointable,
-                                pointableVisitor.getTypeInfo(rtType, out));
-                        break;
-                    case ARRAY:
-                    case MULTISET:
-                        pointable = pointableAllocator.allocateListValue(rtType);
-                        pointable.set(valueReference);
-                        pointableVisitor.visit((AListVisitablePointable) pointable,
-                                pointableVisitor.getTypeInfo(rtType, out));
-                        break;
-                    case MISSING:
-                    case NULL:
-                        if (!visitNull) {
-                            return;
-                        }
-                    default:
-                        pointable = pointableAllocator.allocateFieldValue(rtType);
-                        pointable.set(valueReference);
-                        pointableVisitor.visit((AFlatValuePointable) pointable,
-                                pointableVisitor.getTypeInfo(rtType, out));
-                        break;
-                }
-                break;
-            case MISSING:
-            case NULL:
-                if (!visitNull) {
-                    return;
-                }
-            default:
-                pointable = pointableAllocator.allocateFieldValue(type);
-                pointable.set(valueReference);
-                pointableVisitor.visit((AFlatValuePointable) pointable, pointableVisitor.getTypeInfo(type, out));
-                break;
-        }
-    }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonMessageBuilder.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonMessageBuilder.java
index 5429657..20f8306 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonMessageBuilder.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonMessageBuilder.java
@@ -82,6 +82,16 @@
         buf.put(serAddr);
     }
 
+    public void helloDS(String modulePath) throws IOException {
+        this.type = MessageType.HELO;
+        // sum(string lengths) + 2 from fix array tag and message type
+        dataLength = PythonMessageBuilder.getStringLength(modulePath) + 2;
+        packHeader();
+        MessagePackUtils.packFixArrayHeader(buf, (byte) 2);
+        MessagePackUtils.packStr(buf, "HELLO");
+        MessagePackUtils.packStr(buf, modulePath);
+    }
+
     public void quit() throws HyracksDataException {
         this.type = MessageType.QUIT;
         dataLength = getStringLength("QUIT");
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonTCPSocketProto.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonTCPSocketProto.java
new file mode 100644
index 0000000..7fd3de4
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonTCPSocketProto.java
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.ipc;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.api.IExternalLangIPCProto;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class PythonTCPSocketProto extends AbstractPythonIPCProto
+        implements org.apache.asterix.external.api.IExternalLangIPCProto {
+
+    private final ExternalFunctionResultRouter router;
+    private final Process proc;
+
+    public PythonTCPSocketProto(OutputStream sockOut, ExternalFunctionResultRouter router, Process pythonProc) {
+        super(sockOut);
+        this.router = router;
+        this.proc = pythonProc;
+    }
+
+    @Override
+    public void start() {
+        Pair<Long, Pair<ByteBuffer, Exception>> keyAndBufferBox = router.insertRoute(recvBuffer);
+        this.routeId = keyAndBufferBox.getFirst();
+        this.bufferBox = keyAndBufferBox.getSecond();
+    }
+
+    @Override
+    public void quit() throws HyracksDataException {
+        messageBuilder.quit();
+        router.removeRoute(routeId);
+    }
+
+    @Override
+    public void receiveMsg() throws IOException, AsterixException {
+        Exception except;
+        try {
+            synchronized (bufferBox) {
+                while ((bufferBox.getFirst().limit() == 0 || bufferBox.getSecond() != null) && proc.isAlive()) {
+                    bufferBox.wait(100);
+                }
+            }
+            except = router.getAndRemoveException(routeId);
+            if (!proc.isAlive()) {
+                except = new IOException("Python process exited with code: " + proc.exitValue());
+            }
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw new AsterixException(ErrorCode.EXTERNAL_UDF_EXCEPTION, e);
+        }
+        if (except != null) {
+            throw new AsterixException(except);
+        }
+        if (bufferBox.getFirst() != recvBuffer) {
+            recvBuffer = bufferBox.getFirst();
+        }
+        messageBuilder.readHead(recvBuffer);
+        if (messageBuilder.type == MessageType.ERROR) {
+            unpackerInput.reset(recvBuffer.array(), recvBuffer.position() + recvBuffer.arrayOffset(),
+                    recvBuffer.remaining());
+            unpacker.reset(unpackerInput);
+            throw new AsterixException(unpacker.unpackString());
+        }
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/AbstractLibrarySocketEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/AbstractLibrarySocketEvaluator.java
new file mode 100644
index 0000000..6fcfdcf
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/AbstractLibrarySocketEvaluator.java
@@ -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.
+ */
+package org.apache.asterix.external.library;
+
+import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_UDF_EXCEPTION;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IExternalLangIPCProto;
+import org.apache.asterix.external.api.ILibraryEvaluator;
+import org.apache.asterix.om.functions.IExternalFunctionInfo;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.dataflow.TaskAttemptId;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
+
+public abstract class AbstractLibrarySocketEvaluator extends AbstractStateObject implements ILibraryEvaluator {
+
+    protected IExternalLangIPCProto proto;
+    protected TaskAttemptId task;
+    protected IWarningCollector warningCollector;
+    protected SourceLocation sourceLoc;
+
+    public AbstractLibrarySocketEvaluator(JobId jobId, PythonLibraryEvaluatorId evaluatorId, TaskAttemptId task,
+            IWarningCollector warningCollector, SourceLocation sourceLoc) {
+        super(jobId, evaluatorId);
+        this.task = task;
+        this.warningCollector = warningCollector;
+        this.sourceLoc = sourceLoc;
+    }
+
+    @Override
+    public long initialize(IExternalFunctionInfo finfo) throws IOException, AsterixException {
+        List<String> externalIdents = finfo.getExternalIdentifier();
+        String packageModule = externalIdents.get(0);
+        String clazz;
+        String fn;
+        String externalIdent1 = externalIdents.get(1);
+        int idx = externalIdent1.lastIndexOf('.');
+        if (idx >= 0) {
+            clazz = externalIdent1.substring(0, idx);
+            fn = externalIdent1.substring(idx + 1);
+        } else {
+            clazz = null;
+            fn = externalIdent1;
+        }
+        return proto.init(packageModule, clazz, fn);
+    }
+
+    @Override
+    public ByteBuffer call(long id, IAType[] argTypes, IValueReference[] valueReferences, boolean nullCall)
+            throws IOException {
+        ByteBuffer ret = null;
+        try {
+            ret = proto.call(id, argTypes, valueReferences, nullCall);
+        } catch (AsterixException e) {
+            if (warningCollector.shouldWarn()) {
+                warningCollector.warn(Warning.of(sourceLoc, EXTERNAL_UDF_EXCEPTION, e.getMessage()));
+            }
+        }
+        return ret;
+    }
+
+    @Override
+    public ByteBuffer callMulti(long id, ArrayBackedValueStorage arguments, int numTuples) throws IOException {
+        ByteBuffer ret = null;
+        try {
+            ret = proto.callMulti(id, arguments, numTuples);
+        } catch (AsterixException e) {
+            if (warningCollector.shouldWarn()) {
+                warningCollector.warn(Warning.of(sourceLoc, EXTERNAL_UDF_EXCEPTION, e.getMessage()));
+            }
+        }
+        return ret;
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
index f71150a..050d9f0 100755
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
@@ -49,6 +49,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.Function;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipFile;
 
@@ -77,7 +78,6 @@
 import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.impl.client.HttpClients;
-import org.apache.http.ssl.SSLContexts;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
@@ -92,6 +92,7 @@
 import org.apache.hyracks.control.common.work.AbstractWork;
 import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.ipc.impl.IPCSystem;
+import org.apache.hyracks.ipc.security.NetworkSecurityManager;
 import org.apache.hyracks.ipc.sockets.PlainSocketChannelFactory;
 import org.apache.hyracks.util.file.FileUtil;
 import org.apache.logging.log4j.LogManager;
@@ -102,7 +103,7 @@
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.SerializationFeature;
 
-public final class ExternalLibraryManager implements ILibraryManager, ILifeCycleComponent {
+public class ExternalLibraryManager implements ILibraryManager, ILifeCycleComponent {
 
     public static final String LIBRARY_MANAGER_BASE_DIR_NAME = "library";
 
@@ -140,6 +141,7 @@
     private final ExternalFunctionResultRouter router;
     private final IIOManager ioManager;
     private boolean sslEnabled;
+    private Function<ILibraryManager, CloseableHttpClient> uploadClientSupp;
 
     public ExternalLibraryManager(NodeControllerService ncs, IPersistedResourceRegistry reg, FileReference appDir,
             IIOManager ioManager) {
@@ -155,6 +157,7 @@
         router = new ExternalFunctionResultRouter();
         this.sslEnabled = ncs.getConfiguration().isSslEnabled();
         this.ioManager = ioManager;
+        uploadClientSupp = ExternalLibraryManager::defaultHttpClient;
     }
 
     public void initialize(boolean resetStorageData) throws HyracksDataException {
@@ -486,6 +489,11 @@
         return pythonIPC;
     }
 
+    @Override
+    public NodeControllerService getNcs() {
+        return ncs;
+    }
+
     private static final class DeleteDirectoryWork extends AbstractWork {
 
         private final Path path;
@@ -630,30 +638,37 @@
         }
     }
 
-    //TODO: this should probably be static so it could be reused somewhere else, or made such that the trust store is not
-    // reloaded from disk on every client intialization?
-    private CloseableHttpClient newClient() {
+    public CloseableHttpClient newClient() {
         if (sslEnabled) {
-            try {
-                final INetworkSecurityManager networkSecurityManager = ncs.getNetworkSecurityManager();
-                final INetworkSecurityConfig configuration = networkSecurityManager.getConfiguration();
-                KeyStore trustStore = KeyStore.getInstance(KeyStore.getDefaultType());
-                try (FileInputStream trustStoreFile = new FileInputStream(configuration.getTrustStoreFile())) {
-                    String ksPassword = configuration.getKeyStorePassword();
-                    trustStore.load(trustStoreFile,
-                            ksPassword == null || ksPassword.isEmpty() ? null : ksPassword.toCharArray());
-                }
-                SSLContext sslcontext = SSLContexts.custom().loadTrustMaterial(trustStore, null).build();
-                SSLConnectionSocketFactory sslsf = new SSLConnectionSocketFactory(sslcontext,
-                        new String[] { "TLSv1.2" }, null, SSLConnectionSocketFactory.getDefaultHostnameVerifier());
-                return HttpClients.custom().setSSLSocketFactory(sslsf).build();
-
-            } catch (Exception e) {
-                throw new IllegalStateException(e);
-            }
+            return uploadClientSupp.apply(this);
         } else {
             return HttpClients.createDefault();
         }
     }
 
+    @Override
+    public void setUploadClient(Function<ILibraryManager, CloseableHttpClient> f) {
+        uploadClientSupp = f;
+    }
+
+    private static CloseableHttpClient defaultHttpClient(ILibraryManager extLib) {
+        try {
+            final INetworkSecurityManager networkSecurityManager = extLib.getNcs().getNetworkSecurityManager();
+            final INetworkSecurityConfig configuration = networkSecurityManager.getConfiguration();
+            KeyStore trustStore = KeyStore.getInstance(KeyStore.getDefaultType());
+            try (FileInputStream trustStoreFile = new FileInputStream(configuration.getTrustStoreFile())) {
+                String ksPassword = configuration.getKeyStorePassword();
+                trustStore.load(trustStoreFile,
+                        ksPassword == null || ksPassword.isEmpty() ? null : ksPassword.toCharArray());
+            }
+            SSLContext sslcontext = NetworkSecurityManager.newSSLContext(configuration);
+            SSLConnectionSocketFactory sslsf = new SSLConnectionSocketFactory(sslcontext, new String[] { "TLSv1.2" },
+                    null, SSLConnectionSocketFactory.getDefaultHostnameVerifier());
+            return HttpClients.custom().setSSLSocketFactory(sslsf).build();
+
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java
index 94a4dd2..fb8d761 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java
@@ -28,6 +28,7 @@
 
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.api.ILibraryEvaluator;
 import org.apache.asterix.external.library.msgpack.MessageUnpackerToADM;
 import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.asterix.om.functions.IExternalFunctionInfo;
@@ -49,7 +50,7 @@
 
 class ExternalScalarPythonFunctionEvaluator extends ExternalScalarFunctionEvaluator {
 
-    private final PythonLibraryEvaluator libraryEvaluator;
+    private final ILibraryEvaluator libraryEvaluator;
 
     private final ArrayBackedValueStorage resultBuffer = new ArrayBackedValueStorage();
     private final ByteBuffer argHolder;
@@ -115,7 +116,7 @@
             return;
         }
         try {
-            ByteBuffer res = libraryEvaluator.callPython(fnId, argTypes, argValues, nullCall);
+            ByteBuffer res = libraryEvaluator.call(fnId, argTypes, argValues, nullCall);
             resultBuffer.reset();
             wrap(res, resultBuffer.getDataOutput());
         } catch (Exception e) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryDomainSocketEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryDomainSocketEvaluator.java
new file mode 100644
index 0000000..056aa9a
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryDomainSocketEvaluator.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.external.library;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandle;
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.MethodType;
+import java.lang.invoke.VarHandle;
+import java.net.ProtocolFamily;
+import java.net.SocketAddress;
+import java.net.StandardProtocolFamily;
+import java.nio.channels.Channels;
+import java.nio.channels.SocketChannel;
+import java.nio.file.Path;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.external.ipc.PythonDomainSocketProto;
+import org.apache.asterix.om.functions.IExternalFunctionInfo;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.TaskAttemptId;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.control.common.controllers.NCConfig;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class PythonLibraryDomainSocketEvaluator extends AbstractLibrarySocketEvaluator {
+
+    private final ILibraryManager libMgr;
+    private final Path sockPath;
+    SocketChannel chan;
+    ProcessHandle pid;
+    private static final Logger LOGGER = LogManager.getLogger(ExternalLibraryManager.class);
+
+    public PythonLibraryDomainSocketEvaluator(JobId jobId, PythonLibraryEvaluatorId evaluatorId, ILibraryManager libMgr,
+            TaskAttemptId task, IWarningCollector warningCollector, SourceLocation sourceLoc, Path sockPath) {
+        super(jobId, evaluatorId, task, warningCollector, sourceLoc);
+        this.libMgr = libMgr;
+        this.sockPath = sockPath;
+    }
+
+    public void start() throws IOException, AsterixException {
+        PythonLibraryEvaluatorId fnId = (PythonLibraryEvaluatorId) id;
+        PythonLibrary library =
+                (PythonLibrary) libMgr.getLibrary(fnId.getLibraryDataverseName(), fnId.getLibraryName());
+        String wd = library.getFile().getAbsolutePath();
+        MethodHandles.Lookup lookup = MethodHandles.lookup();
+        SocketAddress sockAddr;
+        try {
+            VarHandle sockEnum = lookup.in(StandardProtocolFamily.class)
+                    .findStaticVarHandle(StandardProtocolFamily.class, "UNIX", StandardProtocolFamily.class);
+            Class domainSock = Class.forName("java.net.UnixDomainSocketAddress");
+            MethodType unixDomainSockAddrType = MethodType.methodType(domainSock, Path.class);
+            MethodHandle unixDomainSockAddr = lookup.findStatic(domainSock, "of", unixDomainSockAddrType);
+            MethodType sockChanMethodType = MethodType.methodType(SocketChannel.class, ProtocolFamily.class);
+            MethodHandle sockChanOpen = lookup.findStatic(SocketChannel.class, "open", sockChanMethodType);
+            sockAddr = ((SocketAddress) unixDomainSockAddr.invoke(sockPath));
+            chan = (SocketChannel) sockChanOpen.invoke(sockEnum.get());
+        } catch (Throwable e) {
+            throw HyracksDataException.create(ErrorCode.LOCAL_NETWORK_ERROR, e);
+        }
+        chan.connect(sockAddr);
+        proto = new PythonDomainSocketProto(Channels.newOutputStream(chan), chan, wd);
+        proto.start();
+        proto.helo();
+        this.pid = ((PythonDomainSocketProto) proto).getPid();
+    }
+
+    @Override
+    public void deallocate() {
+        try {
+            if (proto != null) {
+                proto.quit();
+            }
+            if (chan != null) {
+                chan.close();
+            }
+        } catch (IOException e) {
+            LOGGER.error("Caught exception exiting Python UDF:", e);
+        }
+        if (pid != null && pid.isAlive()) {
+            LOGGER.error("Python UDF " + pid.pid() + " did not exit as expected.");
+        }
+    }
+
+    static PythonLibraryDomainSocketEvaluator getInstance(IExternalFunctionInfo finfo, ILibraryManager libMgr,
+            IHyracksTaskContext ctx, IWarningCollector warningCollector, SourceLocation sourceLoc)
+            throws IOException, AsterixException {
+        PythonLibraryEvaluatorId evaluatorId = new PythonLibraryEvaluatorId(finfo.getLibraryDataverseName(),
+                finfo.getLibraryName(), Thread.currentThread());
+        PythonLibraryDomainSocketEvaluator evaluator =
+                (PythonLibraryDomainSocketEvaluator) ctx.getStateObject(evaluatorId);
+        if (evaluator == null) {
+            Path sockPath = Path.of(ctx.getJobletContext().getServiceContext().getAppConfig()
+                    .getString(NCConfig.Option.PYTHON_DS_PATH));
+            evaluator = new PythonLibraryDomainSocketEvaluator(ctx.getJobletContext().getJobId(), evaluatorId, libMgr,
+                    ctx.getTaskAttemptId(), warningCollector, sourceLoc, sockPath);
+            ctx.getJobletContext().registerDeallocatable(evaluator);
+            evaluator.start();
+            ctx.setStateObject(evaluator);
+        }
+        return evaluator;
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluator.java
deleted file mode 100644
index f82b30d..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluator.java
+++ /dev/null
@@ -1,209 +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.library;
-
-import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_UDF_EXCEPTION;
-import static org.msgpack.core.MessagePack.Code.ARRAY16;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.library.ILibraryManager;
-import org.apache.asterix.external.ipc.ExternalFunctionResultRouter;
-import org.apache.asterix.external.ipc.PythonIPCProto;
-import org.apache.asterix.external.library.msgpack.MessagePackUtils;
-import org.apache.asterix.om.functions.IExternalFunctionInfo;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.types.TypeTagUtil;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.TaskAttemptId;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.SourceLocation;
-import org.apache.hyracks.api.exceptions.Warning;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.resources.IDeallocatable;
-import org.apache.hyracks.data.std.api.IValueReference;
-import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
-import org.apache.hyracks.ipc.impl.IPCSystem;
-
-public class PythonLibraryEvaluator extends AbstractStateObject implements IDeallocatable {
-
-    public static final String ENTRYPOINT = "entrypoint.py";
-    public static final String SITE_PACKAGES = "site-packages";
-
-    private Process p;
-    private ILibraryManager libMgr;
-    private File pythonHome;
-    private PythonIPCProto proto;
-    private ExternalFunctionResultRouter router;
-    private IPCSystem ipcSys;
-    private String sitePkgs;
-    private List<String> pythonArgs;
-    private Map<String, String> pythonEnv;
-    private TaskAttemptId task;
-    private IWarningCollector warningCollector;
-    private SourceLocation sourceLoc;
-
-    public PythonLibraryEvaluator(JobId jobId, PythonLibraryEvaluatorId evaluatorId, ILibraryManager libMgr,
-            File pythonHome, String sitePkgs, List<String> pythonArgs, Map<String, String> pythonEnv,
-            ExternalFunctionResultRouter router, IPCSystem ipcSys, TaskAttemptId task,
-            IWarningCollector warningCollector, SourceLocation sourceLoc) {
-        super(jobId, evaluatorId);
-        this.libMgr = libMgr;
-        this.pythonHome = pythonHome;
-        this.sitePkgs = sitePkgs;
-        this.pythonArgs = pythonArgs;
-        this.pythonEnv = pythonEnv;
-        this.router = router;
-        this.task = task;
-        this.ipcSys = ipcSys;
-        this.warningCollector = warningCollector;
-        this.sourceLoc = sourceLoc;
-    }
-
-    private void initialize() throws IOException, AsterixException {
-        PythonLibraryEvaluatorId fnId = (PythonLibraryEvaluatorId) id;
-        PythonLibrary library =
-                (PythonLibrary) libMgr.getLibrary(fnId.getLibraryDataverseName(), fnId.getLibraryName());
-        String wd = library.getFile().getAbsolutePath();
-        int port = ipcSys.getSocketAddress().getPort();
-        List<String> args = new ArrayList<>();
-        args.add(pythonHome.getAbsolutePath());
-        args.addAll(pythonArgs);
-        args.add(ENTRYPOINT);
-        args.add(InetAddress.getLoopbackAddress().getHostAddress());
-        args.add(Integer.toString(port));
-        args.add(sitePkgs);
-        ProcessBuilder pb = new ProcessBuilder(args.toArray(new String[0]));
-        pb.environment().putAll(pythonEnv);
-        pb.directory(new File(wd));
-        p = pb.start();
-        proto = new PythonIPCProto(p.getOutputStream(), router, p);
-        proto.start();
-        proto.helo();
-    }
-
-    public long initialize(IExternalFunctionInfo finfo) throws IOException, AsterixException {
-        List<String> externalIdents = finfo.getExternalIdentifier();
-        String packageModule = externalIdents.get(0);
-        String clazz;
-        String fn;
-        String externalIdent1 = externalIdents.get(1);
-        int idx = externalIdent1.lastIndexOf('.');
-        if (idx >= 0) {
-            clazz = externalIdent1.substring(0, idx);
-            fn = externalIdent1.substring(idx + 1);
-        } else {
-            clazz = null;
-            fn = externalIdent1;
-        }
-        return proto.init(packageModule, clazz, fn);
-    }
-
-    public ByteBuffer callPython(long id, IAType[] argTypes, IValueReference[] valueReferences, boolean nullCall)
-            throws IOException {
-        ByteBuffer ret = null;
-        try {
-            ret = proto.call(id, argTypes, valueReferences, nullCall);
-        } catch (AsterixException e) {
-            if (warningCollector.shouldWarn()) {
-                warningCollector.warn(Warning.of(sourceLoc, EXTERNAL_UDF_EXCEPTION, e.getMessage()));
-            }
-        }
-        return ret;
-    }
-
-    public ByteBuffer callPythonMulti(long id, ArrayBackedValueStorage arguments, int numTuples) throws IOException {
-        ByteBuffer ret = null;
-        try {
-            ret = proto.callMulti(id, arguments, numTuples);
-        } catch (AsterixException e) {
-            if (warningCollector.shouldWarn()) {
-                warningCollector.warn(Warning.of(sourceLoc, EXTERNAL_UDF_EXCEPTION, e.getMessage()));
-            }
-        }
-        return ret;
-    }
-
-    @Override
-    public void deallocate() {
-        if (p != null) {
-            boolean dead = false;
-            try {
-                p.destroy();
-                dead = p.waitFor(100, TimeUnit.MILLISECONDS);
-            } catch (InterruptedException e) {
-                //gonna kill it anyway
-            }
-            if (!dead) {
-                p.destroyForcibly();
-            }
-        }
-        router.removeRoute(proto.getRouteId());
-    }
-
-    public static ATypeTag peekArgument(IAType type, IValueReference valueReference) throws HyracksDataException {
-        ATypeTag tag = type.getTypeTag();
-        if (tag == ATypeTag.ANY) {
-            TaggedValuePointable pointy = TaggedValuePointable.FACTORY.createPointable();
-            pointy.set(valueReference);
-            ATypeTag rtTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(pointy.getTag());
-            IAType rtType = TypeTagUtil.getBuiltinTypeByTag(rtTypeTag);
-            return MessagePackUtils.peekUnknown(rtType);
-        } else {
-            return MessagePackUtils.peekUnknown(type);
-        }
-    }
-
-    public static void setVoidArgument(ArrayBackedValueStorage argHolder) throws IOException {
-        argHolder.getDataOutput().writeByte(ARRAY16);
-        argHolder.getDataOutput().writeShort((short) 0);
-    }
-
-    public static PythonLibraryEvaluator getInstance(IExternalFunctionInfo finfo, ILibraryManager libMgr,
-            ExternalFunctionResultRouter router, IPCSystem ipcSys, File pythonHome, IHyracksTaskContext ctx,
-            String sitePkgs, List<String> pythonArgs, Map<String, String> pythonEnv, IWarningCollector warningCollector,
-            SourceLocation sourceLoc) throws IOException, AsterixException {
-        PythonLibraryEvaluatorId evaluatorId = new PythonLibraryEvaluatorId(finfo.getLibraryDataverseName(),
-                finfo.getLibraryName(), Thread.currentThread());
-        PythonLibraryEvaluator evaluator = (PythonLibraryEvaluator) ctx.getStateObject(evaluatorId);
-        if (evaluator == null) {
-            evaluator = new PythonLibraryEvaluator(ctx.getJobletContext().getJobId(), evaluatorId, libMgr, pythonHome,
-                    sitePkgs, pythonArgs, pythonEnv, router, ipcSys, ctx.getTaskAttemptId(), warningCollector,
-                    sourceLoc);
-            ctx.getJobletContext().registerDeallocatable(evaluator);
-            evaluator.initialize();
-            ctx.setStateObject(evaluator);
-        }
-        return evaluator;
-    }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluatorFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluatorFactory.java
index 06c9bc9..63a6ec3 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluatorFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluatorFactory.java
@@ -18,10 +18,12 @@
  */
 package org.apache.asterix.external.library;
 
-import static org.apache.asterix.external.library.PythonLibraryEvaluator.SITE_PACKAGES;
+import static org.apache.asterix.external.library.PythonLibraryTCPSocketEvaluator.SITE_PACKAGES;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -31,8 +33,10 @@
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.external.api.ILibraryEvaluator;
 import org.apache.asterix.external.ipc.ExternalFunctionResultRouter;
 import org.apache.asterix.om.functions.IExternalFunctionInfo;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.hyracks.api.config.IApplicationConfig;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -40,83 +44,116 @@
 import org.apache.hyracks.ipc.impl.IPCSystem;
 
 public class PythonLibraryEvaluatorFactory {
-    private final ILibraryManager libraryManager;
-    private final IPCSystem ipcSys;
-    private final File pythonPath;
-    private final IHyracksTaskContext ctx;
-    private final ExternalFunctionResultRouter router;
-    private final String sitePackagesPath;
-    private final List<String> pythonArgs;
-    private final Map<String, String> pythonEnv;
+
+    private ILibraryManager libraryManager;
+    private IPCSystem ipcSys;
+    private File pythonPath;
+    private IHyracksTaskContext ctx;
+    private ExternalFunctionResultRouter router;
+    private String sitePackagesPath;
+    private List<String> pythonArgs;
+    private Map<String, String> pythonEnv;
+
+    private boolean domainSockEnable;
 
     public PythonLibraryEvaluatorFactory(IHyracksTaskContext ctx) throws AsterixException {
         this.ctx = ctx;
+        String dsPath =
+                ctx.getJobletContext().getServiceContext().getAppConfig().getString(NCConfig.Option.PYTHON_DS_PATH);
+        config(dsPath == null ? null : Path.of(dsPath));
         libraryManager = ((INcApplicationContext) ctx.getJobletContext().getServiceContext().getApplicationContext())
                 .getLibraryManager();
-        router = libraryManager.getRouter();
-        ipcSys = libraryManager.getIPCI();
-        IApplicationConfig appCfg = ctx.getJobletContext().getServiceContext().getAppConfig();
-        String pythonPathCmd = appCfg.getString(NCConfig.Option.PYTHON_CMD);
-        boolean findPython = appCfg.getBoolean(NCConfig.Option.PYTHON_CMD_AUTOLOCATE);
-        pythonArgs = new ArrayList<>();
-        if (pythonPathCmd == null) {
-            if (findPython) {
-                //if absolute path to interpreter is not specified, try to use environmental python
-                pythonPathCmd = "/usr/bin/env";
-                pythonArgs.add("python3");
-            } else {
-                throw AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION, "Python interpreter not specified, and "
-                        + NCConfig.Option.PYTHON_CMD_AUTOLOCATE.ini() + " is false");
-            }
-        }
-        pythonEnv = new HashMap<>();
-        String[] envRaw = appCfg.getStringArray((NCConfig.Option.PYTHON_ENV));
-        if (envRaw != null) {
-            for (String rawEnvArg : envRaw) {
-                //TODO: i think equals is shared among all unixes and windows. but it needs verification
-                if (rawEnvArg.length() < 1) {
-                    continue;
-                }
-                String[] rawArgSplit = rawEnvArg.split("(?<!\\\\)=", 2);
-                if (rawArgSplit.length < 2) {
+        if (!domainSockEnable) {
+            router = libraryManager.getRouter();
+            ipcSys = libraryManager.getIPCI();
+            IApplicationConfig appCfg = ctx.getJobletContext().getServiceContext().getAppConfig();
+            String pythonPathCmd = appCfg.getString(NCConfig.Option.PYTHON_CMD);
+            boolean findPython = appCfg.getBoolean(NCConfig.Option.PYTHON_CMD_AUTOLOCATE);
+            pythonArgs = new ArrayList<>();
+            if (pythonPathCmd == null) {
+                if (findPython) {
+                    //if absolute path to interpreter is not specified, try to use environmental python
+                    pythonPathCmd = "/usr/bin/env";
+                    pythonArgs.add("python3");
+                } else {
                     throw AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION,
-                            "Invalid environment variable format detected.");
-                }
-                pythonEnv.put(rawArgSplit[0], rawArgSplit[1]);
-            }
-        }
-        pythonPath = new File(pythonPathCmd);
-        List<String> sitePkgs = new ArrayList<>();
-        sitePkgs.add(SITE_PACKAGES);
-        String[] addlSitePackages = appCfg.getStringArray((NCConfig.Option.PYTHON_ADDITIONAL_PACKAGES));
-        for (String sitePkg : addlSitePackages) {
-            if (sitePkg.length() > 0) {
-                sitePkgs.add(sitePkg);
-            }
-        }
-        if (appCfg.getBoolean(NCConfig.Option.PYTHON_USE_BUNDLED_MSGPACK)) {
-            sitePkgs.add("ipc" + File.separator + SITE_PACKAGES + File.separator);
-        }
-        String[] pythonArgsRaw = appCfg.getStringArray(NCConfig.Option.PYTHON_ARGS);
-        if (pythonArgsRaw != null) {
-            for (String arg : pythonArgsRaw) {
-                if (arg.length() > 0) {
-                    pythonArgs.add(arg);
+                            "Python interpreter not specified or domain socket not found, and "
+                                    + NCConfig.Option.PYTHON_CMD_AUTOLOCATE.ini() + " is false");
                 }
             }
+            pythonEnv = new HashMap<>();
+            String[] envRaw = appCfg.getStringArray((NCConfig.Option.PYTHON_ENV));
+            if (envRaw != null) {
+                for (String rawEnvArg : envRaw) {
+                    //TODO: i think equals is shared among all unixes and windows. but it needs verification
+                    if (rawEnvArg.length() < 1) {
+                        continue;
+                    }
+                    String[] rawArgSplit = rawEnvArg.split("(?<!\\\\)=", 2);
+                    if (rawArgSplit.length < 2) {
+                        throw AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION,
+                                "Invalid environment variable format detected.");
+                    }
+                    pythonEnv.put(rawArgSplit[0], rawArgSplit[1]);
+                }
+            }
+            pythonPath = new File(pythonPathCmd);
+            List<String> sitePkgs = new ArrayList<>();
+            sitePkgs.add(SITE_PACKAGES);
+            String[] addlSitePackages = appCfg.getStringArray((NCConfig.Option.PYTHON_ADDITIONAL_PACKAGES));
+            for (String sitePkg : addlSitePackages) {
+                if (sitePkg.length() > 0) {
+                    sitePkgs.add(sitePkg);
+                }
+            }
+            if (appCfg.getBoolean(NCConfig.Option.PYTHON_USE_BUNDLED_MSGPACK)) {
+                sitePkgs.add("ipc" + File.separator + SITE_PACKAGES + File.separator);
+            }
+            String[] pythonArgsRaw = appCfg.getStringArray(NCConfig.Option.PYTHON_ARGS);
+            if (pythonArgsRaw != null) {
+                for (String arg : pythonArgsRaw) {
+                    if (arg.length() > 0) {
+                        pythonArgs.add(arg);
+                    }
+                }
+            }
+            StringBuilder sitePackagesPathBuilder = new StringBuilder();
+            for (int i = 0; i < sitePkgs.size() - 1; i++) {
+                sitePackagesPathBuilder.append(sitePkgs.get(i));
+                sitePackagesPathBuilder.append(File.pathSeparator);
+            }
+            sitePackagesPathBuilder.append(sitePkgs.get(sitePkgs.size() - 1));
+            sitePackagesPath = sitePackagesPathBuilder.toString();
         }
-        StringBuilder sitePackagesPathBuilder = new StringBuilder();
-        for (int i = 0; i < sitePkgs.size() - 1; i++) {
-            sitePackagesPathBuilder.append(sitePkgs.get(i));
-            sitePackagesPathBuilder.append(File.pathSeparator);
-        }
-        sitePackagesPathBuilder.append(sitePkgs.get(sitePkgs.size() - 1));
-        sitePackagesPath = sitePackagesPathBuilder.toString();
     }
 
-    public PythonLibraryEvaluator getEvaluator(IExternalFunctionInfo fnInfo, SourceLocation sourceLoc)
+    public ILibraryEvaluator getEvaluator(IExternalFunctionInfo fnInfo, SourceLocation sourceLoc)
             throws IOException, AsterixException {
-        return PythonLibraryEvaluator.getInstance(fnInfo, libraryManager, router, ipcSys, pythonPath, ctx,
-                sitePackagesPath, pythonArgs, pythonEnv, ctx.getWarningCollector(), sourceLoc);
+        if (domainSockEnable) {
+            return PythonLibraryDomainSocketEvaluator.getInstance(fnInfo, libraryManager, ctx,
+                    ctx.getWarningCollector(), sourceLoc);
+        } else {
+            return PythonLibraryTCPSocketEvaluator.getInstance(fnInfo, libraryManager, router, ipcSys, pythonPath, ctx,
+                    sitePackagesPath, pythonArgs, pythonEnv, ctx.getWarningCollector(), sourceLoc);
+        }
+    }
+
+    private void config(Path sockPath) throws AsterixException {
+        if (sockPath == null) {
+            domainSockEnable = false;
+            return;
+        }
+        Runtime rt = Runtime.getRuntime();
+        if (rt.version().feature() >= 17 && SystemUtils.IS_OS_LINUX) {
+            if (Files.exists(sockPath)) {
+                domainSockEnable = true;
+            } else {
+                throw AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION,
+                        "Domain socket was not found at specified path");
+            }
+        } else {
+            throw AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION,
+                    "Domain socket path specified, but Java version is below 17 or OS is not Linux");
+        }
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryTCPSocketEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryTCPSocketEvaluator.java
new file mode 100644
index 0000000..385d738
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryTCPSocketEvaluator.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.library;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.external.ipc.ExternalFunctionResultRouter;
+import org.apache.asterix.external.ipc.PythonTCPSocketProto;
+import org.apache.asterix.om.functions.IExternalFunctionInfo;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.TaskAttemptId;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.ipc.impl.IPCSystem;
+
+public class PythonLibraryTCPSocketEvaluator extends AbstractLibrarySocketEvaluator {
+
+    public static final String ENTRYPOINT = "entrypoint.py";
+    public static final String SITE_PACKAGES = "site-packages";
+
+    private Process p;
+    private ILibraryManager libMgr;
+    private File pythonHome;
+    private ExternalFunctionResultRouter router;
+    private IPCSystem ipcSys;
+    private String sitePkgs;
+    private List<String> pythonArgs;
+    private Map<String, String> pythonEnv;
+
+    public PythonLibraryTCPSocketEvaluator(JobId jobId, PythonLibraryEvaluatorId evaluatorId, ILibraryManager libMgr,
+            File pythonHome, String sitePkgs, List<String> pythonArgs, Map<String, String> pythonEnv,
+            ExternalFunctionResultRouter router, IPCSystem ipcSys, TaskAttemptId task,
+            IWarningCollector warningCollector, SourceLocation sourceLoc) {
+        super(jobId, evaluatorId, task, warningCollector, sourceLoc);
+        this.libMgr = libMgr;
+        this.pythonHome = pythonHome;
+        this.sitePkgs = sitePkgs;
+        this.pythonArgs = pythonArgs;
+        this.pythonEnv = pythonEnv;
+        this.router = router;
+        this.ipcSys = ipcSys;
+    }
+
+    @Override
+    public void start() throws IOException, AsterixException {
+        PythonLibraryEvaluatorId fnId = (PythonLibraryEvaluatorId) id;
+        PythonLibrary library =
+                (PythonLibrary) libMgr.getLibrary(fnId.getLibraryDataverseName(), fnId.getLibraryName());
+        String wd = library.getFile().getAbsolutePath();
+        int port = ipcSys.getSocketAddress().getPort();
+        List<String> args = new ArrayList<>();
+        args.add(pythonHome.getAbsolutePath());
+        args.addAll(pythonArgs);
+        args.add(ENTRYPOINT);
+        args.add(InetAddress.getLoopbackAddress().getHostAddress());
+        args.add(Integer.toString(port));
+        args.add(sitePkgs);
+        ProcessBuilder pb = new ProcessBuilder(args.toArray(new String[0]));
+        pb.environment().putAll(pythonEnv);
+        pb.directory(new File(wd));
+        p = pb.start();
+        proto = new PythonTCPSocketProto(p.getOutputStream(), router, p);
+        proto.start();
+        proto.helo();
+    }
+
+    @Override
+    public void deallocate() {
+        if (p != null) {
+            boolean dead = false;
+            try {
+                p.destroy();
+                dead = p.waitFor(100, TimeUnit.MILLISECONDS);
+            } catch (InterruptedException e) {
+                //gonna kill it anyway
+            }
+            if (!dead) {
+                p.destroyForcibly();
+            }
+        }
+        router.removeRoute(proto.getRouteId());
+    }
+
+    static PythonLibraryTCPSocketEvaluator getInstance(IExternalFunctionInfo finfo, ILibraryManager libMgr,
+            ExternalFunctionResultRouter router, IPCSystem ipcSys, File pythonHome, IHyracksTaskContext ctx,
+            String sitePkgs, List<String> pythonArgs, Map<String, String> pythonEnv, IWarningCollector warningCollector,
+            SourceLocation sourceLoc) throws IOException, AsterixException {
+        PythonLibraryEvaluatorId evaluatorId = new PythonLibraryEvaluatorId(finfo.getLibraryDataverseName(),
+                finfo.getLibraryName(), Thread.currentThread());
+        PythonLibraryTCPSocketEvaluator evaluator = (PythonLibraryTCPSocketEvaluator) ctx.getStateObject(evaluatorId);
+        if (evaluator == null) {
+            evaluator = new PythonLibraryTCPSocketEvaluator(ctx.getJobletContext().getJobId(), evaluatorId, libMgr,
+                    pythonHome, sitePkgs, pythonArgs, pythonEnv, router, ipcSys, ctx.getTaskAttemptId(),
+                    warningCollector, sourceLoc);
+            ctx.getJobletContext().registerDeallocatable(evaluator);
+            evaluator.start();
+            ctx.setStateObject(evaluator);
+        }
+        return evaluator;
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MsgPackAccessors.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MsgPackAccessors.java
index a90a183..6efbb6e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MsgPackAccessors.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MsgPackAccessors.java
@@ -124,7 +124,7 @@
             int s = pointable.getStartOffset();
             int i = AInt32SerializerDeserializer.getInt(b, s + 1);
             out.writeByte(INT32);
-            out.writeByte(i);
+            out.writeInt(i);
             return null;
         }
     }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java
index 741dad2..5f8a3f0 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java
@@ -33,12 +33,13 @@
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.external.ipc.PythonIPCProto;
-import org.apache.asterix.external.library.PythonLibraryEvaluator;
+import org.apache.asterix.external.api.IExternalLangIPCProto;
+import org.apache.asterix.external.api.ILibraryEvaluator;
 import org.apache.asterix.external.library.PythonLibraryEvaluatorFactory;
 import org.apache.asterix.external.library.msgpack.MessageUnpackerToADM;
 import org.apache.asterix.external.library.msgpack.MsgPackPointableVisitor;
 import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.asterix.om.functions.IExternalFunctionDescriptor;
 import org.apache.asterix.om.pointables.PointableAllocator;
 import org.apache.asterix.om.types.ATypeTag;
@@ -50,6 +51,7 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -87,7 +89,7 @@
             private ArrayBackedValueStorage outputWrapper;
             private List<ArrayBackedValueStorage> argHolders;
             ArrayTupleBuilder tupleBuilder;
-            private List<Pair<Long, PythonLibraryEvaluator>> libraryEvaluators;
+            private List<Pair<Long, ILibraryEvaluator>> libraryEvaluators;
             private ATypeTag[][] nullCalls;
             private int[] numCalls;
             private VoidPointable ref;
@@ -97,6 +99,7 @@
             private MessageUnpackerToADM unpackerToADM;
             private PointableAllocator pointableAllocator;
             private MsgPackPointableVisitor pointableVisitor;
+            private TaggedValuePointable anyPointer;
 
             @Override
             public void open() throws HyracksDataException {
@@ -109,7 +112,7 @@
                 try {
                     PythonLibraryEvaluatorFactory evalFactory = new PythonLibraryEvaluatorFactory(ctx);
                     for (IExternalFunctionDescriptor fnDesc : fnDescs) {
-                        PythonLibraryEvaluator eval = evalFactory.getEvaluator(fnDesc.getFunctionInfo(), sourceLoc);
+                        ILibraryEvaluator eval = evalFactory.getEvaluator(fnDesc.getFunctionInfo(), sourceLoc);
                         long id = eval.initialize(fnDesc.getFunctionInfo());
                         libraryEvaluators.add(new Pair<>(id, eval));
                     }
@@ -133,6 +136,7 @@
                 unpackerToADM = new MessageUnpackerToADM();
                 pointableAllocator = new PointableAllocator();
                 pointableVisitor = new MsgPackPointableVisitor();
+                anyPointer = TaggedValuePointable.FACTORY.createPointable();
             }
 
             private void resetBuffers(int numTuples, int[] numCalls) {
@@ -177,8 +181,12 @@
                             int numEntries = unpacker.unpackArrayHeader();
                             for (int j = 0; j < numEntries; j++) {
                                 if (ctx.getWarningCollector().shouldWarn()) {
-                                    ctx.getWarningCollector().warn(Warning.of(sourceLoc,
-                                            ErrorCode.EXTERNAL_UDF_EXCEPTION, unpacker.unpackString()));
+                                    //TODO: in domain socket mode, a NUL can appear at the end of the stacktrace strings.
+                                    //      this should probably not happen but warnings with control characters should
+                                    //      also be properly escaped
+                                    ctx.getWarningCollector()
+                                            .warn(Warning.of(sourceLoc, ErrorCode.EXTERNAL_UDF_EXCEPTION,
+                                                    unpacker.unpackString().replace('\0', ' ')));
                                 }
                             }
                         } catch (MessagePackException e) {
@@ -211,8 +219,8 @@
                                 for (int colIdx = 0; colIdx < cols.length; colIdx++) {
                                     ref.set(buffer.array(), tRef.getFieldStart(cols[colIdx]),
                                             tRef.getFieldLength(cols[colIdx]));
-                                    ATypeTag argumentPresence = PythonLibraryEvaluator
-                                            .peekArgument(fnDescs[func].getArgumentTypes()[colIdx], ref);
+                                    ATypeTag argumentPresence = ExternalDataUtils
+                                            .peekArgument(fnDescs[func].getArgumentTypes()[colIdx], ref, anyPointer);
                                     argumentStatus = handleNullMatrix(func, t, argumentPresence, argumentStatus);
                                 }
                             }
@@ -224,7 +232,7 @@
                                 for (int colIdx = 0; colIdx < cols.length; colIdx++) {
                                     ref.set(buffer.array(), tRef.getFieldStart(cols[colIdx]),
                                             tRef.getFieldLength(cols[colIdx]));
-                                    PythonIPCProto.visitValueRef(fnDescs[func].getArgumentTypes()[colIdx],
+                                    IExternalLangIPCProto.visitValueRef(fnDescs[func].getArgumentTypes()[colIdx],
                                             argHolders.get(func).getDataOutput(), ref, pointableAllocator,
                                             pointableVisitor, fnDescs[func].getFunctionInfo().getNullCall());
                                 }
@@ -232,21 +240,25 @@
                                 numCalls[func]--;
                             }
                             if (cols.length == 0) {
-                                PythonLibraryEvaluator.setVoidArgument(argHolders.get(func));
+                                ExternalDataUtils.setVoidArgument(argHolders.get(func));
                             }
                         }
                     }
 
                     //TODO: maybe this could be done in parallel for each unique library evaluator?
                     for (int argHolderIdx = 0; argHolderIdx < argHolders.size(); argHolderIdx++) {
-                        Pair<Long, PythonLibraryEvaluator> fnEval = libraryEvaluators.get(argHolderIdx);
-                        ByteBuffer columnResult = fnEval.getSecond().callPythonMulti(fnEval.getFirst(),
+                        Pair<Long, ILibraryEvaluator> fnEval = libraryEvaluators.get(argHolderIdx);
+                        ByteBuffer columnResult = fnEval.getSecond().callMulti(fnEval.getFirst(),
                                 argHolders.get(argHolderIdx), numCalls[argHolderIdx]);
                         if (columnResult != null) {
                             Pair<ByteBuffer, Counter> resultholder = batchResults.get(argHolderIdx);
-                            if (resultholder.getFirst().capacity() < columnResult.capacity()) {
-                                ByteBuffer realloc = ctx.reallocateFrame(resultholder.getFirst(),
-                                        columnResult.capacity() * 2, false);
+                            if (resultholder.getFirst().capacity() < columnResult.remaining()) {
+                                ByteBuffer realloc =
+                                        ctx.reallocateFrame(resultholder.getFirst(),
+                                                ctx.getInitialFrameSize()
+                                                        * ((columnResult.remaining() / ctx.getInitialFrameSize()) + 1),
+                                                false);
+                                realloc.limit(columnResult.limit());
                                 resultholder.setFirst(realloc);
                             }
                             ByteBuffer resultBuf = resultholder.getFirst();
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalScanOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalScanOperatorDescriptor.java
index 82b8113..0250ba8 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalScanOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalScanOperatorDescriptor.java
@@ -27,6 +27,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.NoOpOperatorStats;
 import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
 import org.apache.hyracks.storage.am.common.api.ITupleFilter;
@@ -66,23 +67,18 @@
 
         return new AbstractUnaryOutputSourceOperatorNodePushable() {
 
-            private IOperatorStats stats;
+            private IOperatorStats stats = NoOpOperatorStats.INSTANCE;
 
             @Override
             public void initialize() throws HyracksDataException {
                 IDataSourceAdapter adapter;
-                if (ctx.getStatsCollector() != null) {
-                    stats = ctx.getStatsCollector().getOrAddOperatorStats(getDisplayName());
-                }
                 try {
                     writer.open();
                     ITupleFilter tupleFilter =
                             tupleFilterFactory != null ? tupleFilterFactory.createTupleFilter(ctx) : null;
                     adapter = adapterFactory.createAdapter(ctx, partition);
                     adapter.start(partition, writer, tupleFilter, outputLimit);
-                    if (stats != null) {
-                        stats.getTupleCounter().update(adapter.getProcessedTuples());
-                    }
+                    stats.getInputTupleCounter().update(adapter.getProcessedTuples());
                 } catch (Exception e) {
                     writer.fail();
                     throw HyracksDataException.create(e);
@@ -90,6 +86,11 @@
                     writer.close();
                 }
             }
+
+            @Override
+            public void setOperatorStats(IOperatorStats stats) {
+                this.stats = stats;
+            }
         };
 
     }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
index f0b9c90..429706e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
@@ -333,107 +333,4 @@
          */
         public static final Set<String> VALID_TIME_ZONES = Set.of(TimeZone.getAvailableIDs());
     }
-
-    public static class AwsS3 {
-        private AwsS3() {
-            throw new AssertionError("do not instantiate");
-        }
-
-        public static final String REGION_FIELD_NAME = "region";
-        public static final String ACCESS_KEY_ID_FIELD_NAME = "accessKeyId";
-        public static final String SECRET_ACCESS_KEY_FIELD_NAME = "secretAccessKey";
-        public static final String SESSION_TOKEN_FIELD_NAME = "sessionToken";
-        public static final String SERVICE_END_POINT_FIELD_NAME = "serviceEndpoint";
-
-        // AWS S3 specific error codes
-        public static final String ERROR_INTERNAL_ERROR = "InternalError";
-        public static final String ERROR_SLOW_DOWN = "SlowDown";
-        public static final String ERROR_METHOD_NOT_IMPLEMENTED = "NotImplemented";
-
-        public static boolean isRetryableError(String errorCode) {
-            return errorCode.equals(ERROR_INTERNAL_ERROR) || errorCode.equals(ERROR_SLOW_DOWN);
-        }
-
-        /*
-         * Hadoop-AWS
-         * AWS connectors for s3 and s3n are deprecated.
-         */
-        public static final String HADOOP_ACCESS_KEY_ID = "fs.s3a.access.key";
-        public static final String HADOOP_SECRET_ACCESS_KEY = "fs.s3a.secret.key";
-        public static final String HADOOP_SESSION_TOKEN = "fs.s3a.session.token";
-        public static final String HADOOP_REGION = "fs.s3a.region";
-        public static final String HADOOP_SERVICE_END_POINT = "fs.s3a.endpoint";
-
-        /*
-         * Internal configurations
-         */
-        //Allows accessing directories as file system path
-        public static final String HADOOP_PATH_STYLE_ACCESS = "fs.s3a.path.style.access";
-        //The number of maximum HTTP connections in connection pool
-        public static final String HADOOP_S3_CONNECTION_POOL_SIZE = "fs.s3a.connection.maximum";
-        //S3 used protocol
-        public static final String HADOOP_S3_PROTOCOL = "s3a";
-
-        //Hadoop credentials provider key
-        public static final String HADOOP_CREDENTIAL_PROVIDER_KEY = "fs.s3a.aws.credentials.provider";
-        //Anonymous credential provider
-        public static final String HADOOP_ANONYMOUS_ACCESS = "org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider";
-        //Temporary credential provider
-        public static final String HADOOP_TEMP_ACCESS = "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
-
-    }
-
-    /*
-     * Note: Azure Blob and Azure Datalake use identical authentication, so they are using the same properties.
-     * If they end up diverging, then properties for AzureBlob and AzureDataLake need to be created.
-     */
-    public static class Azure {
-        private Azure() {
-            throw new AssertionError("do not instantiate");
-        }
-
-        /*
-         * Asterix Configuration Keys
-         */
-        public static final String MANAGED_IDENTITY_ID_FIELD_NAME = "managedIdentityId";
-        public static final String ACCOUNT_NAME_FIELD_NAME = "accountName";
-        public static final String ACCOUNT_KEY_FIELD_NAME = "accountKey";
-        public static final String SHARED_ACCESS_SIGNATURE_FIELD_NAME = "sharedAccessSignature";
-        public static final String TENANT_ID_FIELD_NAME = "tenantId";
-        public static final String CLIENT_ID_FIELD_NAME = "clientId";
-        public static final String CLIENT_SECRET_FIELD_NAME = "clientSecret";
-        public static final String CLIENT_CERTIFICATE_FIELD_NAME = "clientCertificate";
-        public static final String CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME = "clientCertificatePassword";
-        public static final String ENDPOINT_FIELD_NAME = "endpoint";
-
-        // Specific Azure data lake property
-        /*
-        The behavior of Data Lake (true file system) is to read the files of the specified prefix only, example:
-        storage/myData/personal/file1.json
-        storage/myData/personal/file2.json
-        storage/myData/file3.json
-        If the prefix used is "myData", then only the file file3.json is read. However, if the property "recursive"
-        is set to "true" when creating the external dataset, then it goes recursively overall the paths, and the result
-        is file1.json, file2.json and file3.json.
-         */
-        public static final String RECURSIVE_FIELD_NAME = "recursive";
-
-        /*
-         * Hadoop-Azure
-         */
-        //Used when accountName and accessKey are provided
-        public static final String HADOOP_AZURE_FS_ACCOUNT_KEY = "fs.azure.account.key";
-        //Used when a connectionString is provided
-        public static final String HADOOP_AZURE_FS_SAS = "fs.azure.sas";
-        public static final String HADOOP_AZURE_BLOB_PROTOCOL = "wasbs";
-        public static final String HADOOP_AZURE_DATALAKE_PROTOCOL = "abfss";
-    }
-
-    public static class GCS {
-        private GCS() {
-            throw new AssertionError("do not instantiate");
-        }
-
-        public static final String JSON_CREDENTIALS_FIELD_NAME = "jsonCredentials";
-    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
index 8e38eed..5bf5844 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
@@ -18,43 +18,9 @@
  */
 package org.apache.asterix.external.util;
 
-import static com.google.cloud.storage.Storage.BlobListOption;
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_SOURCE_ERROR;
 import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_REQ_PARAM_VAL;
 import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_NOT_ALLOWED_AT_SAME_TIME;
 import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_REQUIRED;
-import static org.apache.asterix.common.exceptions.ErrorCode.PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT;
-import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
-import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT;
-import static org.apache.asterix.common.exceptions.ErrorCode.S3_REGION_NOT_SUPPORTED;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ACCESS_KEY_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ERROR_METHOD_NOT_IMPLEMENTED;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_ACCESS_KEY_ID;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_ANONYMOUS_ACCESS;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_CREDENTIAL_PROVIDER_KEY;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_PATH_STYLE_ACCESS;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_S3_CONNECTION_POOL_SIZE;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_S3_PROTOCOL;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_SECRET_ACCESS_KEY;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_SESSION_TOKEN;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_TEMP_ACCESS;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ACCOUNT_KEY_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ACCOUNT_NAME_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_CERTIFICATE_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_SECRET_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ENDPOINT_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_BLOB_PROTOCOL;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_FS_ACCOUNT_KEY;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_FS_SAS;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.MANAGED_IDENTITY_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.RECURSIVE_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.SHARED_ACCESS_SIGNATURE_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.TENANT_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.GCS.JSON_CREDENTIALS_FIELD_NAME;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ADAPTER_NAME_GCS;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_DELIMITER;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ESCAPE;
@@ -65,18 +31,15 @@
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_QUOTE;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_END;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_START;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.validateAzureBlobProperties;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.validateAzureDataLakeProperties;
+import static org.apache.asterix.external.util.google.gcs.GCSUtils.validateProperties;
 import static org.apache.asterix.runtime.evaluators.functions.StringEvaluatorUtils.RESERVED_REGEX_CHARS;
-import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+import static org.msgpack.core.MessagePack.Code.ARRAY16;
 
-import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.net.URI;
-import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Base64;
@@ -85,7 +48,6 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
-import java.util.function.BiPredicate;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.regex.PatternSyntaxException;
@@ -105,21 +67,25 @@
 import org.apache.asterix.external.api.IRecordReaderFactory;
 import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
 import org.apache.asterix.external.library.JavaLibrary;
+import org.apache.asterix.external.library.msgpack.MessagePackUtils;
 import org.apache.asterix.external.util.ExternalDataConstants.ParquetOptions;
+import org.apache.asterix.external.util.aws.s3.S3Utils;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeTagUtil;
 import org.apache.asterix.runtime.evaluators.common.NumberUtils;
 import org.apache.asterix.runtime.projection.DataProjectionInfo;
 import org.apache.asterix.runtime.projection.FunctionCallInformation;
-import org.apache.hadoop.fs.s3a.Constants;
-import org.apache.hadoop.mapred.JobConf;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 import org.apache.hyracks.api.exceptions.SourceLocation;
-import org.apache.hyracks.api.exceptions.Warning;
-import org.apache.hyracks.api.util.CleanupUtils;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.parsers.BooleanParserFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.FloatParserFactory;
@@ -129,46 +95,6 @@
 import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 import org.apache.hyracks.util.StorageUtil;
 
-import com.azure.core.credential.AzureSasCredential;
-import com.azure.core.http.rest.PagedIterable;
-import com.azure.identity.ClientCertificateCredentialBuilder;
-import com.azure.identity.ClientSecretCredentialBuilder;
-import com.azure.identity.ManagedIdentityCredentialBuilder;
-import com.azure.storage.blob.BlobContainerClient;
-import com.azure.storage.blob.BlobServiceClient;
-import com.azure.storage.blob.BlobServiceClientBuilder;
-import com.azure.storage.blob.models.BlobItem;
-import com.azure.storage.blob.models.ListBlobsOptions;
-import com.azure.storage.common.StorageSharedKeyCredential;
-import com.azure.storage.common.policy.RequestRetryOptions;
-import com.azure.storage.file.datalake.DataLakeFileSystemClient;
-import com.azure.storage.file.datalake.DataLakeServiceClient;
-import com.azure.storage.file.datalake.DataLakeServiceClientBuilder;
-import com.azure.storage.file.datalake.models.ListPathsOptions;
-import com.azure.storage.file.datalake.models.PathItem;
-import com.google.api.gax.paging.Page;
-import com.google.auth.oauth2.ServiceAccountCredentials;
-import com.google.cloud.storage.Blob;
-import com.google.cloud.storage.Storage;
-import com.google.cloud.storage.StorageOptions;
-
-import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
-import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
-import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
-import software.amazon.awssdk.core.exception.SdkException;
-import software.amazon.awssdk.regions.Region;
-import software.amazon.awssdk.services.s3.S3Client;
-import software.amazon.awssdk.services.s3.S3ClientBuilder;
-import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
-import software.amazon.awssdk.services.s3.model.ListObjectsResponse;
-import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
-import software.amazon.awssdk.services.s3.model.ListObjectsV2Response;
-import software.amazon.awssdk.services.s3.model.S3Exception;
-import software.amazon.awssdk.services.s3.model.S3Object;
-import software.amazon.awssdk.services.s3.model.S3Response;
-
 public class ExternalDataUtils {
     private static final Map<ATypeTag, IValueParserFactory> valueParserFactoryMap = new EnumMap<>(ATypeTag.class);
     private static final int DEFAULT_MAX_ARGUMENT_SZ = 1024 * 1024;
@@ -468,7 +394,8 @@
     /**
      * Fills the configuration of the external dataset and its adapter with default values if not provided by user.
      *
-     * @param configuration external data configuration
+     * @param configuration
+     *            external data configuration
      */
     public static void defaultConfiguration(Map<String, String> configuration) {
         String format = configuration.get(ExternalDataConstants.KEY_FORMAT);
@@ -490,8 +417,10 @@
      * Prepares the configuration of the external data and its adapter by filling the information required by
      * adapters and parsers.
      *
-     * @param adapterName   adapter name
-     * @param configuration external data configuration
+     * @param adapterName
+     *            adapter name
+     * @param configuration
+     *            external data configuration
      */
     public static void prepare(String adapterName, Map<String, String> configuration) {
         if (!configuration.containsKey(ExternalDataConstants.KEY_READER)) {
@@ -513,7 +442,8 @@
      * Normalizes the values of certain parameters of the adapter configuration. This should happen before persisting
      * the metadata (e.g. when creating external datasets or feeds) and when creating an adapter factory.
      *
-     * @param configuration external data configuration
+     * @param configuration
+     *            external data configuration
      */
     public static void normalize(Map<String, String> configuration) {
         // normalize the "format" parameter
@@ -533,8 +463,10 @@
     /**
      * Validates the parameter values of the adapter configuration. This should happen after normalizing the values.
      *
-     * @param configuration external data configuration
-     * @throws HyracksDataException HyracksDataException
+     * @param configuration
+     *            external data configuration
+     * @throws HyracksDataException
+     *             HyracksDataException
      */
     public static void validate(Map<String, String> configuration) throws HyracksDataException {
         String format = configuration.get(ExternalDataConstants.KEY_FORMAT);
@@ -596,7 +528,8 @@
      * Validates adapter specific external dataset properties. Specific properties for different adapters should be
      * validated here
      *
-     * @param configuration properties
+     * @param configuration
+     *            properties
      */
     public static void validateAdapterSpecificProperties(Map<String, String> configuration, SourceLocation srcLoc,
             IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
@@ -604,16 +537,16 @@
 
         switch (type) {
             case ExternalDataConstants.KEY_ADAPTER_NAME_AWS_S3:
-                AwsS3.validateProperties(configuration, srcLoc, collector);
+                S3Utils.validateProperties(configuration, srcLoc, collector);
                 break;
             case ExternalDataConstants.KEY_ADAPTER_NAME_AZURE_BLOB:
-                Azure.validateAzureBlobProperties(configuration, srcLoc, collector, appCtx);
+                validateAzureBlobProperties(configuration, srcLoc, collector, appCtx);
                 break;
             case ExternalDataConstants.KEY_ADAPTER_NAME_AZURE_DATA_LAKE:
-                Azure.validateAzureDataLakeProperties(configuration, srcLoc, collector, appCtx);
+                validateAzureDataLakeProperties(configuration, srcLoc, collector, appCtx);
                 break;
-            case KEY_ADAPTER_NAME_GCS:
-                GCS.validateProperties(configuration, srcLoc, collector);
+            case ExternalDataConstants.KEY_ADAPTER_NAME_GCS:
+                validateProperties(configuration, srcLoc, collector);
                 break;
             default:
                 // Nothing needs to be done
@@ -624,7 +557,8 @@
     /**
      * Regex matches all the provided patterns against the provided path
      *
-     * @param path path to check against
+     * @param path
+     *            path to check against
      * @return {@code true} if all patterns match, {@code false} otherwise
      */
     public static boolean matchPatterns(List<Matcher> matchers, String path) {
@@ -639,7 +573,8 @@
     /**
      * Converts the wildcard to proper regex
      *
-     * @param pattern wildcard pattern to convert
+     * @param pattern
+     *            wildcard pattern to convert
      * @return regex expression
      */
     public static String patternToRegex(String pattern) {
@@ -728,7 +663,8 @@
     /**
      * Adjusts the prefix (if needed) and returns it
      *
-     * @param configuration configuration
+     * @param configuration
+     *            configuration
      */
     public static String getPrefix(Map<String, String> configuration) {
         return getPrefix(configuration, true);
@@ -743,8 +679,10 @@
     }
 
     /**
-     * @param configuration configuration map
-     * @throws CompilationException Compilation exception
+     * @param configuration
+     *            configuration map
+     * @throws CompilationException
+     *             Compilation exception
      */
     public static void validateIncludeExclude(Map<String, String> configuration) throws CompilationException {
         // Ensure that include and exclude are not provided at the same time + ensure valid format or property
@@ -828,8 +766,10 @@
     /**
      * Validate Parquet dataset's declared type and configuration
      *
-     * @param properties        external dataset configuration
-     * @param datasetRecordType dataset declared type
+     * @param properties
+     *            external dataset configuration
+     * @param datasetRecordType
+     *            dataset declared type
      */
     public static void validateParquetTypeAndConfiguration(Map<String, String> properties,
             ARecordType datasetRecordType) throws CompilationException {
@@ -844,7 +784,7 @@
         }
     }
 
-    private static boolean isParquetFormat(Map<String, String> properties) {
+    public static boolean isParquetFormat(Map<String, String> properties) {
         String inputFormat = properties.get(ExternalDataConstants.KEY_INPUT_FORMAT);
         return ExternalDataConstants.CLASS_NAME_PARQUET_INPUT_FORMAT.equals(inputFormat)
                 || ExternalDataConstants.INPUT_FORMAT_PARQUET.equals(inputFormat)
@@ -862,7 +802,8 @@
     /**
      * Serialize {@link ARecordType} as Base64 string to pass it to {@link org.apache.hadoop.conf.Configuration}
      *
-     * @param expectedType expected type
+     * @param expectedType
+     *            expected type
      * @return the expected type as Base64 string
      */
     private static String serializeExpectedTypeToString(ARecordType expectedType) throws IOException {
@@ -881,7 +822,8 @@
      * Serialize {@link FunctionCallInformation} map as Base64 string to pass it to
      * {@link org.apache.hadoop.conf.Configuration}
      *
-     * @param functionCallInfoMap function information map
+     * @param functionCallInfoMap
+     *            function information map
      * @return function information map as Base64 string
      */
     static String serializeFunctionCallInfoToString(Map<String, FunctionCallInformation> functionCallInfoMap)
@@ -893,1008 +835,6 @@
         return encoder.encodeToString(byteArrayOutputStream.toByteArray());
     }
 
-    public static class AwsS3 {
-        private AwsS3() {
-            throw new AssertionError("do not instantiate");
-        }
-
-        /**
-         * Builds the S3 client using the provided configuration
-         *
-         * @param configuration properties
-         * @return S3 client
-         * @throws CompilationException CompilationException
-         */
-        public static S3Client buildAwsS3Client(Map<String, String> configuration) throws CompilationException {
-            // TODO(Hussain): Need to ensure that all required parameters are present in a previous step
-            String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
-            String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
-            String sessionToken = configuration.get(ExternalDataConstants.AwsS3.SESSION_TOKEN_FIELD_NAME);
-            String regionId = configuration.get(ExternalDataConstants.AwsS3.REGION_FIELD_NAME);
-            String serviceEndpoint = configuration.get(ExternalDataConstants.AwsS3.SERVICE_END_POINT_FIELD_NAME);
-
-            S3ClientBuilder builder = S3Client.builder();
-
-            // Credentials
-            AwsCredentialsProvider credentialsProvider;
-
-            // No auth required
-            if (accessKeyId == null) {
-                credentialsProvider = AnonymousCredentialsProvider.create();
-            } else {
-                // auth required, check for temporary or permanent credentials
-                if (sessionToken != null) {
-                    credentialsProvider = StaticCredentialsProvider
-                            .create(AwsSessionCredentials.create(accessKeyId, secretAccessKey, sessionToken));
-                } else {
-                    credentialsProvider =
-                            StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKeyId, secretAccessKey));
-                }
-            }
-
-            builder.credentialsProvider(credentialsProvider);
-
-            // Validate the region
-            List<Region> regions = S3Client.serviceMetadata().regions();
-            Optional<Region> selectedRegion =
-                    regions.stream().filter(region -> region.id().equals(regionId)).findFirst();
-
-            if (selectedRegion.isEmpty()) {
-                throw new CompilationException(S3_REGION_NOT_SUPPORTED, regionId);
-            }
-            builder.region(selectedRegion.get());
-
-            // Validate the service endpoint if present
-            if (serviceEndpoint != null) {
-                try {
-                    URI uri = new URI(serviceEndpoint);
-                    try {
-                        builder.endpointOverride(uri);
-                    } catch (NullPointerException ex) {
-                        throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-                    }
-                } catch (URISyntaxException ex) {
-                    throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR,
-                            String.format("Invalid service endpoint %s", serviceEndpoint));
-                }
-            }
-
-            return builder.build();
-        }
-
-        /**
-         * Builds the S3 client using the provided configuration
-         *
-         * @param configuration      properties
-         * @param numberOfPartitions number of partitions in the cluster
-         */
-        public static void configureAwsS3HdfsJobConf(JobConf conf, Map<String, String> configuration,
-                int numberOfPartitions) {
-            String accessKeyId = configuration.get(ExternalDataConstants.AwsS3.ACCESS_KEY_ID_FIELD_NAME);
-            String secretAccessKey = configuration.get(ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME);
-            String sessionToken = configuration.get(ExternalDataConstants.AwsS3.SESSION_TOKEN_FIELD_NAME);
-            String serviceEndpoint = configuration.get(ExternalDataConstants.AwsS3.SERVICE_END_POINT_FIELD_NAME);
-
-            //Disable caching S3 FileSystem
-            HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_S3_PROTOCOL);
-
-            /*
-             * Authentication Methods:
-             * 1- Anonymous: no accessKeyId and no secretAccessKey
-             * 2- Temporary: has to provide accessKeyId, secretAccessKey and sessionToken
-             * 3- Private: has to provide accessKeyId and secretAccessKey
-             */
-            if (accessKeyId == null) {
-                //Tells hadoop-aws it is an anonymous access
-                conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_ANONYMOUS_ACCESS);
-            } else {
-                conf.set(HADOOP_ACCESS_KEY_ID, accessKeyId);
-                conf.set(HADOOP_SECRET_ACCESS_KEY, secretAccessKey);
-                if (sessionToken != null) {
-                    conf.set(HADOOP_SESSION_TOKEN, sessionToken);
-                    //Tells hadoop-aws it is a temporary access
-                    conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_TEMP_ACCESS);
-                }
-            }
-
-            /*
-             * This is to allow S3 definition to have path-style form. Should always be true to match the current
-             * way we access files in S3
-             */
-            conf.set(HADOOP_PATH_STYLE_ACCESS, ExternalDataConstants.TRUE);
-
-            /*
-             * Set the size of S3 connection pool to be the number of partitions
-             */
-            conf.set(HADOOP_S3_CONNECTION_POOL_SIZE, String.valueOf(numberOfPartitions));
-
-            if (serviceEndpoint != null) {
-                // Validation of the URL should be done at hadoop-aws level
-                conf.set(ExternalDataConstants.AwsS3.HADOOP_SERVICE_END_POINT, serviceEndpoint);
-            } else {
-                //Region is ignored and buckets could be found by the central endpoint
-                conf.set(ExternalDataConstants.AwsS3.HADOOP_SERVICE_END_POINT, Constants.CENTRAL_ENDPOINT);
-            }
-        }
-
-        /**
-         * Validate external dataset properties
-         *
-         * @param configuration properties
-         * @throws CompilationException Compilation exception
-         */
-        public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
-                IWarningCollector collector) throws CompilationException {
-
-            // check if the format property is present
-            if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
-                throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
-            }
-
-            // Both parameters should be passed, or neither should be passed (for anonymous/no auth)
-            String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
-            String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
-            if (accessKeyId == null || secretAccessKey == null) {
-                // If one is passed, the other is required
-                if (accessKeyId != null) {
-                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
-                            ACCESS_KEY_ID_FIELD_NAME);
-                } else if (secretAccessKey != null) {
-                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
-                            SECRET_ACCESS_KEY_FIELD_NAME);
-                }
-            }
-
-            validateIncludeExclude(configuration);
-
-            // Check if the bucket is present
-            S3Client s3Client = buildAwsS3Client(configuration);
-            S3Response response;
-            boolean useOldApi = false;
-            String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-            String prefix = getPrefix(configuration);
-
-            try {
-                response = isBucketEmpty(s3Client, container, prefix, false);
-            } catch (S3Exception ex) {
-                // Method not implemented, try falling back to old API
-                try {
-                    // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
-                    if (ex.awsErrorDetails().errorCode().equals(ERROR_METHOD_NOT_IMPLEMENTED)) {
-                        useOldApi = true;
-                        response = isBucketEmpty(s3Client, container, prefix, true);
-                    } else {
-                        throw ex;
-                    }
-                } catch (SdkException ex2) {
-                    throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-                }
-            } catch (SdkException ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            } finally {
-                if (s3Client != null) {
-                    CleanupUtils.close(s3Client, null);
-                }
-            }
-
-            boolean isEmpty = useOldApi ? ((ListObjectsResponse) response).contents().isEmpty()
-                    : ((ListObjectsV2Response) response).contents().isEmpty();
-            if (isEmpty && collector.shouldWarn()) {
-                Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
-                collector.warn(warning);
-            }
-
-            // Returns 200 only in case the bucket exists, otherwise, throws an exception. However, to
-            // ensure coverage, check if the result is successful as well and not only catch exceptions
-            if (!response.sdkHttpResponse().isSuccessful()) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_CONTAINER_NOT_FOUND, container);
-            }
-        }
-
-        /**
-         * Checks for a single object in the specified bucket to determine if the bucket is empty or not.
-         *
-         * @param s3Client  s3 client
-         * @param container the container name
-         * @param prefix    Prefix to be used
-         * @param useOldApi flag whether to use the old API or not
-         * @return returns the S3 response
-         */
-        private static S3Response isBucketEmpty(S3Client s3Client, String container, String prefix, boolean useOldApi) {
-            S3Response response;
-            if (useOldApi) {
-                ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder();
-                listObjectsBuilder.prefix(prefix);
-                response = s3Client.listObjects(listObjectsBuilder.bucket(container).maxKeys(1).build());
-            } else {
-                ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder();
-                listObjectsBuilder.prefix(prefix);
-                response = s3Client.listObjectsV2(listObjectsBuilder.bucket(container).maxKeys(1).build());
-            }
-            return response;
-        }
-
-        /**
-         * Returns the lists of S3 objects.
-         *
-         * @param configuration         properties
-         * @param includeExcludeMatcher include/exclude matchers to apply
-         */
-        public static List<S3Object> listS3Objects(Map<String, String> configuration,
-                IncludeExcludeMatcher includeExcludeMatcher, IWarningCollector warningCollector)
-                throws CompilationException {
-            // Prepare to retrieve the objects
-            List<S3Object> filesOnly;
-            String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-            S3Client s3Client = buildAwsS3Client(configuration);
-            String prefix = getPrefix(configuration);
-
-            try {
-                filesOnly = listS3Objects(s3Client, container, prefix, includeExcludeMatcher);
-            } catch (S3Exception ex) {
-                // New API is not implemented, try falling back to old API
-                try {
-                    // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
-                    if (ex.awsErrorDetails().errorCode()
-                            .equals(ExternalDataConstants.AwsS3.ERROR_METHOD_NOT_IMPLEMENTED)) {
-                        filesOnly = oldApiListS3Objects(s3Client, container, prefix, includeExcludeMatcher);
-                    } else {
-                        throw ex;
-                    }
-                } catch (SdkException ex2) {
-                    throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-                }
-            } catch (SdkException ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            } finally {
-                if (s3Client != null) {
-                    CleanupUtils.close(s3Client, null);
-                }
-            }
-
-            // Warn if no files are returned
-            if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
-                Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
-                warningCollector.warn(warning);
-            }
-
-            return filesOnly;
-        }
-
-        /**
-         * Uses the latest API to retrieve the objects from the storage.
-         *
-         * @param s3Client              S3 client
-         * @param container             container name
-         * @param prefix                definition prefix
-         * @param includeExcludeMatcher include/exclude matchers to apply
-         */
-        private static List<S3Object> listS3Objects(S3Client s3Client, String container, String prefix,
-                IncludeExcludeMatcher includeExcludeMatcher) {
-            String newMarker = null;
-            List<S3Object> filesOnly = new ArrayList<>();
-
-            ListObjectsV2Response listObjectsResponse;
-            ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder().bucket(container);
-            listObjectsBuilder.prefix(prefix);
-
-            while (true) {
-                // List the objects from the start, or from the last marker in case of truncated result
-                if (newMarker == null) {
-                    listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.build());
-                } else {
-                    listObjectsResponse =
-                            s3Client.listObjectsV2(listObjectsBuilder.continuationToken(newMarker).build());
-                }
-
-                // Collect the paths to files only
-                collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
-                        includeExcludeMatcher.getMatchersList(), filesOnly);
-
-                // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
-                if (!listObjectsResponse.isTruncated()) {
-                    break;
-                } else {
-                    newMarker = listObjectsResponse.nextContinuationToken();
-                }
-            }
-
-            return filesOnly;
-        }
-
-        /**
-         * Uses the old API (in case the new API is not implemented) to retrieve the objects from the storage
-         *
-         * @param s3Client              S3 client
-         * @param container             container name
-         * @param prefix                definition prefix
-         * @param includeExcludeMatcher include/exclude matchers to apply
-         */
-        private static List<S3Object> oldApiListS3Objects(S3Client s3Client, String container, String prefix,
-                IncludeExcludeMatcher includeExcludeMatcher) {
-            String newMarker = null;
-            List<S3Object> filesOnly = new ArrayList<>();
-
-            ListObjectsResponse listObjectsResponse;
-            ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder().bucket(container);
-            listObjectsBuilder.prefix(prefix);
-
-            while (true) {
-                // List the objects from the start, or from the last marker in case of truncated result
-                if (newMarker == null) {
-                    listObjectsResponse = s3Client.listObjects(listObjectsBuilder.build());
-                } else {
-                    listObjectsResponse = s3Client.listObjects(listObjectsBuilder.marker(newMarker).build());
-                }
-
-                // Collect the paths to files only
-                collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
-                        includeExcludeMatcher.getMatchersList(), filesOnly);
-
-                // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
-                if (!listObjectsResponse.isTruncated()) {
-                    break;
-                } else {
-                    newMarker = listObjectsResponse.nextMarker();
-                }
-            }
-
-            return filesOnly;
-        }
-
-        /**
-         * AWS S3 returns all the objects as paths, not differentiating between folder and files. The path is considered
-         * a file if it does not end up with a "/" which is the separator in a folder structure.
-         *
-         * @param s3Objects List of returned objects
-         */
-        private static void collectAndFilterFiles(List<S3Object> s3Objects,
-                BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<S3Object> filesOnly) {
-            for (S3Object object : s3Objects) {
-                // skip folders
-                if (object.key().endsWith("/")) {
-                    continue;
-                }
-
-                // No filter, add file
-                if (predicate.test(matchers, object.key())) {
-                    filesOnly.add(object);
-                }
-            }
-        }
-    }
-
-    /*
-     * Note: Azure Blob and Azure Datalake use identical authentication, so they are using the same properties.
-     * If they end up diverging, then properties for AzureBlob and AzureDataLake need to be created.
-     */
-    public static class Azure {
-        private Azure() {
-            throw new AssertionError("do not instantiate");
-        }
-
-        /**
-         * Builds the Azure storage account using the provided configuration
-         *
-         * @param configuration properties
-         * @return client
-         */
-        public static BlobServiceClient buildAzureBlobClient(IApplicationContext appCtx,
-                Map<String, String> configuration) throws CompilationException {
-            String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
-            String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
-            String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
-            String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
-            String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
-            String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
-            String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
-            String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
-            String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
-            String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
-
-            // Client builder
-            BlobServiceClientBuilder builder = new BlobServiceClientBuilder();
-            int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
-            RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
-            builder.retryOptions(requestRetryOptions);
-
-            // Endpoint is required
-            if (endpoint == null) {
-                throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
-            }
-            builder.endpoint(endpoint);
-
-            // Shared Key
-            if (accountName != null || accountKey != null) {
-                if (accountName == null) {
-                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
-                            ACCOUNT_KEY_FIELD_NAME);
-                }
-
-                if (accountKey == null) {
-                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
-                            ACCOUNT_NAME_FIELD_NAME);
-                }
-
-                Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
-                        MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
-                        CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
-                if (provided.isPresent()) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
-                            ACCOUNT_KEY_FIELD_NAME);
-                }
-                StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
-                builder.credential(credential);
-            }
-
-            // Shared access signature
-            if (sharedAccessSignature != null) {
-                Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
-                        CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
-                        CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
-                if (provided.isPresent()) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
-                            SHARED_ACCESS_SIGNATURE_FIELD_NAME);
-                }
-                AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
-                builder.credential(credential);
-            }
-
-            // Managed Identity auth
-            if (managedIdentityId != null) {
-                Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME,
-                        CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME,
-                        TENANT_ID_FIELD_NAME);
-                if (provided.isPresent()) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
-                            MANAGED_IDENTITY_ID_FIELD_NAME);
-                }
-                builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
-            }
-
-            // Client secret & certificate auth
-            if (clientId != null) {
-                // Both (or neither) client secret and client secret were provided, only one is allowed
-                if ((clientSecret == null) == (clientCertificate == null)) {
-                    if (clientSecret != null) {
-                        throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
-                                CLIENT_CERTIFICATE_FIELD_NAME);
-                    } else {
-                        throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
-                                CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
-                    }
-                }
-
-                // Tenant ID is required
-                if (tenantId == null) {
-                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
-                            CLIENT_ID_FIELD_NAME);
-                }
-
-                // Client certificate password is not allowed if client secret is used
-                if (clientCertificatePassword != null && clientSecret != null) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
-                            CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
-                }
-
-                // Use AD authentication
-                if (clientSecret != null) {
-                    ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
-                    secret.clientId(clientId);
-                    secret.tenantId(tenantId);
-                    secret.clientSecret(clientSecret);
-                    builder.credential(secret.build());
-                } else {
-                    // Certificate
-                    ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
-                    certificate.clientId(clientId);
-                    certificate.tenantId(tenantId);
-                    try {
-                        InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
-                        if (clientCertificatePassword == null) {
-                            Method pemCertificate = ClientCertificateCredentialBuilder.class
-                                    .getDeclaredMethod("pemCertificate", InputStream.class);
-                            pemCertificate.setAccessible(true);
-                            pemCertificate.invoke(certificate, certificateContent);
-                        } else {
-                            Method pemCertificate = ClientCertificateCredentialBuilder.class
-                                    .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
-                            pemCertificate.setAccessible(true);
-                            pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
-                        }
-                    } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
-                        throw new CompilationException(EXTERNAL_SOURCE_ERROR, ex.getMessage());
-                    }
-                    builder.credential(certificate.build());
-                }
-            }
-
-            // If client id is not present, ensure client secret, certificate, tenant id and client certificate
-            // password are not present
-            if (clientId == null) {
-                Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
-                        CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
-                if (provided.isPresent()) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
-                            SHARED_ACCESS_SIGNATURE_FIELD_NAME);
-                }
-            }
-
-            try {
-                return builder.buildClient();
-            } catch (Exception ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            }
-        }
-
-        /**
-         * Builds the Azure data lake storage account using the provided configuration
-         *
-         * @param configuration properties
-         * @return client
-         */
-        public static DataLakeServiceClient buildAzureDatalakeClient(IApplicationContext appCtx,
-                Map<String, String> configuration) throws CompilationException {
-            String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
-            String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
-            String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
-            String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
-            String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
-            String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
-            String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
-            String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
-            String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
-            String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
-
-            // Client builder
-            DataLakeServiceClientBuilder builder = new DataLakeServiceClientBuilder();
-            int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
-            RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
-            builder.retryOptions(requestRetryOptions);
-
-            // Endpoint is required
-            if (endpoint == null) {
-                throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
-            }
-            builder.endpoint(endpoint);
-
-            // Shared Key
-            if (accountName != null || accountKey != null) {
-                if (accountName == null) {
-                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
-                            ACCOUNT_KEY_FIELD_NAME);
-                }
-
-                if (accountKey == null) {
-                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
-                            ACCOUNT_NAME_FIELD_NAME);
-                }
-
-                Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
-                        MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
-                        CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
-                if (provided.isPresent()) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
-                            ACCOUNT_KEY_FIELD_NAME);
-                }
-                StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
-                builder.credential(credential);
-            }
-
-            // Shared access signature
-            if (sharedAccessSignature != null) {
-                Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
-                        CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
-                        CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
-                if (provided.isPresent()) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
-                            SHARED_ACCESS_SIGNATURE_FIELD_NAME);
-                }
-                AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
-                builder.credential(credential);
-            }
-
-            // Managed Identity auth
-            if (managedIdentityId != null) {
-                Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME,
-                        CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME,
-                        TENANT_ID_FIELD_NAME);
-                if (provided.isPresent()) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
-                            MANAGED_IDENTITY_ID_FIELD_NAME);
-                }
-                builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
-            }
-
-            // Client secret & certificate auth
-            if (clientId != null) {
-                // Both (or neither) client secret and client secret were provided, only one is allowed
-                if ((clientSecret == null) == (clientCertificate == null)) {
-                    if (clientSecret != null) {
-                        throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
-                                CLIENT_CERTIFICATE_FIELD_NAME);
-                    } else {
-                        throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
-                                CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
-                    }
-                }
-
-                // Tenant ID is required
-                if (tenantId == null) {
-                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
-                            CLIENT_ID_FIELD_NAME);
-                }
-
-                // Client certificate password is not allowed if client secret is used
-                if (clientCertificatePassword != null && clientSecret != null) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
-                            CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
-                }
-
-                // Use AD authentication
-                if (clientSecret != null) {
-                    ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
-                    secret.clientId(clientId);
-                    secret.tenantId(tenantId);
-                    secret.clientSecret(clientSecret);
-                    builder.credential(secret.build());
-                } else {
-                    // Certificate
-                    ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
-                    certificate.clientId(clientId);
-                    certificate.tenantId(tenantId);
-                    try {
-                        InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
-                        if (clientCertificatePassword == null) {
-                            Method pemCertificate = ClientCertificateCredentialBuilder.class
-                                    .getDeclaredMethod("pemCertificate", InputStream.class);
-                            pemCertificate.setAccessible(true);
-                            pemCertificate.invoke(certificate, certificateContent);
-                        } else {
-                            Method pemCertificate = ClientCertificateCredentialBuilder.class
-                                    .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
-                            pemCertificate.setAccessible(true);
-                            pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
-                        }
-                    } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
-                        throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-                    }
-                    builder.credential(certificate.build());
-                }
-            }
-
-            // If client id is not present, ensure client secret, certificate, tenant id and client certificate
-            // password are not present
-            if (clientId == null) {
-                Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
-                        CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
-                if (provided.isPresent()) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
-                            SHARED_ACCESS_SIGNATURE_FIELD_NAME);
-                }
-            }
-
-            try {
-                return builder.buildClient();
-            } catch (Exception ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            }
-        }
-
-        public static List<BlobItem> listBlobItems(BlobServiceClient blobServiceClient,
-                Map<String, String> configuration, IncludeExcludeMatcher includeExcludeMatcher,
-                IWarningCollector warningCollector) throws CompilationException {
-            String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-
-            List<BlobItem> filesOnly = new ArrayList<>();
-
-            // Ensure the validity of include/exclude
-            ExternalDataUtils.validateIncludeExclude(configuration);
-
-            BlobContainerClient blobContainer;
-            try {
-                blobContainer = blobServiceClient.getBlobContainerClient(container);
-
-                // Get all objects in a container and extract the paths to files
-                ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
-                listBlobsOptions.setPrefix(ExternalDataUtils.getPrefix(configuration));
-                Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
-
-                // Collect the paths to files only
-                collectAndFilterBlobFiles(blobItems, includeExcludeMatcher.getPredicate(),
-                        includeExcludeMatcher.getMatchersList(), filesOnly);
-
-                // Warn if no files are returned
-                if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
-                    Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
-                    warningCollector.warn(warning);
-                }
-            } catch (Exception ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            }
-
-            return filesOnly;
-        }
-
-        /**
-         * Collects and filters the files only, and excludes any folders
-         *
-         * @param items     storage items
-         * @param predicate predicate to test with for file filtration
-         * @param matchers  include/exclude matchers to test against
-         * @param filesOnly List containing the files only (excluding folders)
-         */
-        private static void collectAndFilterBlobFiles(Iterable<BlobItem> items,
-                BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<BlobItem> filesOnly) {
-            for (BlobItem item : items) {
-                String uri = item.getName();
-
-                // skip folders
-                if (uri.endsWith("/")) {
-                    continue;
-                }
-
-                // No filter, add file
-                if (predicate.test(matchers, uri)) {
-                    filesOnly.add(item);
-                }
-            }
-        }
-
-        public static List<PathItem> listDatalakePathItems(DataLakeServiceClient client,
-                Map<String, String> configuration, IncludeExcludeMatcher includeExcludeMatcher,
-                IWarningCollector warningCollector) throws CompilationException {
-            String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-
-            List<PathItem> filesOnly = new ArrayList<>();
-
-            // Ensure the validity of include/exclude
-            ExternalDataUtils.validateIncludeExclude(configuration);
-
-            DataLakeFileSystemClient fileSystemClient;
-            try {
-                fileSystemClient = client.getFileSystemClient(container);
-
-                // Get all objects in a container and extract the paths to files
-                ListPathsOptions listOptions = new ListPathsOptions();
-                boolean recursive = Boolean.parseBoolean(configuration.get(RECURSIVE_FIELD_NAME));
-                listOptions.setRecursive(recursive);
-                listOptions.setPath(ExternalDataUtils.getPrefix(configuration, false));
-                PagedIterable<PathItem> pathItems = fileSystemClient.listPaths(listOptions, null);
-
-                // Collect the paths to files only
-                collectAndFilterDatalakeFiles(pathItems, includeExcludeMatcher.getPredicate(),
-                        includeExcludeMatcher.getMatchersList(), filesOnly);
-
-                // Warn if no files are returned
-                if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
-                    Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
-                    warningCollector.warn(warning);
-                }
-            } catch (Exception ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            }
-
-            return filesOnly;
-        }
-
-        /**
-         * Collects and filters the files only, and excludes any folders
-         *
-         * @param items     storage items
-         * @param predicate predicate to test with for file filtration
-         * @param matchers  include/exclude matchers to test against
-         * @param filesOnly List containing the files only (excluding folders)
-         */
-        private static void collectAndFilterDatalakeFiles(Iterable<PathItem> items,
-                BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<PathItem> filesOnly) {
-            for (PathItem item : items) {
-                String uri = item.getName();
-
-                // skip folders
-                if (uri.endsWith("/")) {
-                    continue;
-                }
-
-                // No filter, add file
-                if (predicate.test(matchers, uri)) {
-                    filesOnly.add(item);
-                }
-            }
-        }
-
-        /**
-         * Validate external dataset properties
-         *
-         * @param configuration properties
-         * @throws CompilationException Compilation exception
-         */
-        public static void validateAzureBlobProperties(Map<String, String> configuration, SourceLocation srcLoc,
-                IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
-
-            // check if the format property is present
-            if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
-                throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
-            }
-
-            validateIncludeExclude(configuration);
-
-            // Check if the bucket is present
-            BlobServiceClient blobServiceClient;
-            try {
-                String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-                blobServiceClient = buildAzureBlobClient(appCtx, configuration);
-                BlobContainerClient blobContainer = blobServiceClient.getBlobContainerClient(container);
-
-                // Get all objects in a container and extract the paths to files
-                ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
-                listBlobsOptions.setPrefix(getPrefix(configuration));
-                Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
-
-                if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
-                    Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
-                    collector.warn(warning);
-                }
-            } catch (CompilationException ex) {
-                throw ex;
-            } catch (Exception ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            }
-        }
-
-        /**
-         * Validate external dataset properties
-         *
-         * @param configuration properties
-         * @throws CompilationException Compilation exception
-         */
-        public static void validateAzureDataLakeProperties(Map<String, String> configuration, SourceLocation srcLoc,
-                IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
-
-            // check if the format property is present
-            if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
-                throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
-            }
-
-            validateIncludeExclude(configuration);
-
-            // Check if the bucket is present
-            DataLakeServiceClient dataLakeServiceClient;
-            try {
-                String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-                dataLakeServiceClient = buildAzureDatalakeClient(appCtx, configuration);
-                DataLakeFileSystemClient fileSystemClient = dataLakeServiceClient.getFileSystemClient(container);
-
-                // Get all objects in a container and extract the paths to files
-                ListPathsOptions listPathsOptions = new ListPathsOptions();
-                listPathsOptions.setPath(getPrefix(configuration));
-                Iterable<PathItem> blobItems = fileSystemClient.listPaths(listPathsOptions, null);
-
-                if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
-                    Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
-                    collector.warn(warning);
-                }
-            } catch (CompilationException ex) {
-                throw ex;
-            } catch (Exception ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            }
-        }
-
-        /**
-         * Builds the Azure Blob storage client using the provided configuration
-         *
-         * @param configuration properties
-         * @see <a href="https://docs.microsoft.com/en-us/azure/databricks/data/data-sources/azure/azure-storage">Azure
-         * Blob storage</a>
-         */
-        public static void configureAzureHdfsJobConf(JobConf conf, Map<String, String> configuration, String endPoint) {
-            String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-            String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
-            String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
-
-            //Disable caching S3 FileSystem
-            HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_AZURE_BLOB_PROTOCOL);
-
-            //Key for Hadoop configuration
-            StringBuilder hadoopKey = new StringBuilder();
-            //Value for Hadoop configuration
-            String hadoopValue;
-            if (accountKey != null || sharedAccessSignature != null) {
-                if (accountKey != null) {
-                    hadoopKey.append(HADOOP_AZURE_FS_ACCOUNT_KEY).append('.');
-                    //Set only the AccountKey
-                    hadoopValue = accountKey;
-                } else {
-                    //Use SAS for Hadoop FS as connectionString is provided
-                    hadoopKey.append(HADOOP_AZURE_FS_SAS).append('.');
-                    //Setting the container is required for SAS
-                    hadoopKey.append(container).append('.');
-                    //Set the connection string for SAS
-                    hadoopValue = sharedAccessSignature;
-                }
-                //Set the endPoint, which includes the AccountName
-                hadoopKey.append(endPoint);
-                //Tells Hadoop we are reading from Blob Storage
-                conf.set(hadoopKey.toString(), hadoopValue);
-            }
-        }
-    }
-
-    public static class GCS {
-        private GCS() {
-            throw new AssertionError("do not instantiate");
-
-        }
-
-        //TODO(htowaileb): Add validation step similar to other externals, which also checks if empty bucket
-        //upon creating the external dataset
-
-        /**
-         * Builds the client using the provided configuration
-         *
-         * @param configuration properties
-         * @return clientasterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
-         * @throws CompilationException CompilationException
-         */
-        public static Storage buildClient(Map<String, String> configuration) throws CompilationException {
-            String jsonCredentials = configuration.get(JSON_CREDENTIALS_FIELD_NAME);
-
-            StorageOptions.Builder builder = StorageOptions.newBuilder();
-
-            // Use credentials if available
-            if (jsonCredentials != null) {
-                try (InputStream credentialsStream = new ByteArrayInputStream(jsonCredentials.getBytes())) {
-                    builder.setCredentials(ServiceAccountCredentials.fromStream(credentialsStream));
-                } catch (IOException ex) {
-                    throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-                }
-            }
-
-            return builder.build().getService();
-        }
-
-        /**
-         * Validate external dataset properties
-         *
-         * @param configuration properties
-         * @throws CompilationException Compilation exception
-         */
-        public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
-                IWarningCollector collector) throws CompilationException {
-
-            // check if the format property is present
-            if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
-                throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
-            }
-
-            // parquet is not supported for google cloud storage
-            if (isParquetFormat(configuration)) {
-                throw new CompilationException(INVALID_REQ_PARAM_VAL, srcLoc, KEY_FORMAT,
-                        configuration.get(KEY_FORMAT));
-            }
-
-            validateIncludeExclude(configuration);
-            String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-
-            try {
-                BlobListOption limitOption = BlobListOption.pageSize(1);
-                BlobListOption prefixOption = BlobListOption.prefix(getPrefix(configuration));
-                Storage storage = buildClient(configuration);
-                Page<Blob> items = storage.list(container, limitOption, prefixOption);
-
-                if (!items.iterateAll().iterator().hasNext() && collector.shouldWarn()) {
-                    Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
-                    collector.warn(warning);
-                }
-            } catch (CompilationException ex) {
-                throw ex;
-            } catch (Exception ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            }
-        }
-    }
-
     public static int roundUpToNearestFrameSize(int size, int framesize) {
         return ((size / framesize) + 1) * framesize;
     }
@@ -1911,7 +851,25 @@
         return maxArgSz;
     }
 
-    private static Optional<String> getFirstNotNull(Map<String, String> configuration, String... parameters) {
+    public static Optional<String> getFirstNotNull(Map<String, String> configuration, String... parameters) {
         return Arrays.stream(parameters).filter(field -> configuration.get(field) != null).findFirst();
     }
+
+    public static ATypeTag peekArgument(IAType type, IValueReference valueReference, TaggedValuePointable pointy)
+            throws HyracksDataException {
+        ATypeTag tag = type.getTypeTag();
+        if (tag == ATypeTag.ANY) {
+            pointy.set(valueReference);
+            ATypeTag rtTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(pointy.getTag());
+            IAType rtType = TypeTagUtil.getBuiltinTypeByTag(rtTypeTag);
+            return MessagePackUtils.peekUnknown(rtType);
+        } else {
+            return MessagePackUtils.peekUnknown(type);
+        }
+    }
+
+    public static void setVoidArgument(ArrayBackedValueStorage argHolder) throws IOException {
+        argHolder.getDataOutput().writeByte(ARRAY16);
+        argHolder.getDataOutput().writeShort((short) 0);
+    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.java
new file mode 100644
index 0000000..79bbbe2
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.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.external.util.aws.s3;
+
+public class S3Constants {
+    private S3Constants() {
+        throw new AssertionError("do not instantiate");
+    }
+
+    public static final String REGION_FIELD_NAME = "region";
+    public static final String INSTANCE_PROFILE_FIELD_NAME = "instanceProfile";
+    public static final String ACCESS_KEY_ID_FIELD_NAME = "accessKeyId";
+    public static final String SECRET_ACCESS_KEY_FIELD_NAME = "secretAccessKey";
+    public static final String SESSION_TOKEN_FIELD_NAME = "sessionToken";
+    public static final String SERVICE_END_POINT_FIELD_NAME = "serviceEndpoint";
+
+    // AWS S3 specific error codes
+    public static final String ERROR_INTERNAL_ERROR = "InternalError";
+    public static final String ERROR_SLOW_DOWN = "SlowDown";
+    public static final String ERROR_METHOD_NOT_IMPLEMENTED = "NotImplemented";
+
+    /*
+     * Hadoop-AWS
+     * AWS connectors for s3 and s3n are deprecated.
+     */
+    public static final String HADOOP_ACCESS_KEY_ID = "fs.s3a.access.key";
+    public static final String HADOOP_SECRET_ACCESS_KEY = "fs.s3a.secret.key";
+    public static final String HADOOP_SESSION_TOKEN = "fs.s3a.session.token";
+    public static final String HADOOP_REGION = "fs.s3a.region";
+    public static final String HADOOP_SERVICE_END_POINT = "fs.s3a.endpoint";
+
+    /*
+     * Internal configurations
+     */
+    //Allows accessing directories as file system path
+    public static final String HADOOP_PATH_STYLE_ACCESS = "fs.s3a.path.style.access";
+    //The number of maximum HTTP connections in connection pool
+    public static final String HADOOP_S3_CONNECTION_POOL_SIZE = "fs.s3a.connection.maximum";
+    //S3 used protocol
+    public static final String HADOOP_S3_PROTOCOL = "s3a";
+
+    //Hadoop credentials provider key
+    public static final String HADOOP_CREDENTIAL_PROVIDER_KEY = "fs.s3a.aws.credentials.provider";
+    //Anonymous credential provider
+    public static final String HADOOP_ANONYMOUS_ACCESS = "org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider";
+    //Temporary credential provider
+    public static final String HADOOP_TEMP_ACCESS = "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Utils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Utils.java
new file mode 100644
index 0000000..6775bf12b
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Utils.java
@@ -0,0 +1,475 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.util.aws.s3;
+
+import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_PARAM_VALUE_ALLOWED_VALUE;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.S3_REGION_NOT_SUPPORTED;
+import static org.apache.asterix.external.util.ExternalDataUtils.getPrefix;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.*;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiPredicate;
+import java.util.regex.Matcher;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.api.util.CleanupUtils;
+
+import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+import software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.exception.SdkException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.S3ClientBuilder;
+import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
+import software.amazon.awssdk.services.s3.model.ListObjectsResponse;
+import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
+import software.amazon.awssdk.services.s3.model.ListObjectsV2Response;
+import software.amazon.awssdk.services.s3.model.S3Exception;
+import software.amazon.awssdk.services.s3.model.S3Object;
+import software.amazon.awssdk.services.s3.model.S3Response;
+
+public class S3Utils {
+    private S3Utils() {
+        throw new AssertionError("do not instantiate");
+    }
+
+    public static boolean isRetryableError(String errorCode) {
+        return errorCode.equals(ERROR_INTERNAL_ERROR) || errorCode.equals(ERROR_SLOW_DOWN);
+    }
+
+    /**
+     * Builds the S3 client using the provided configuration
+     *
+     * @param configuration properties
+     * @return S3 client
+     * @throws CompilationException CompilationException
+     */
+    public static S3Client buildAwsS3Client(Map<String, String> configuration) throws CompilationException {
+        // TODO(Hussain): Need to ensure that all required parameters are present in a previous step
+        String instanceProfile = configuration.get(INSTANCE_PROFILE_FIELD_NAME);
+        String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+        String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+        String sessionToken = configuration.get(SESSION_TOKEN_FIELD_NAME);
+        String regionId = configuration.get(REGION_FIELD_NAME);
+        String serviceEndpoint = configuration.get(SERVICE_END_POINT_FIELD_NAME);
+
+        S3ClientBuilder builder = S3Client.builder();
+
+        // Credentials
+        AwsCredentialsProvider credentialsProvider;
+
+        // nothing provided, anonymous authentication
+        if (instanceProfile == null && accessKeyId == null && secretAccessKey == null && sessionToken == null) {
+            credentialsProvider = AnonymousCredentialsProvider.create();
+        } else if (instanceProfile != null) {
+
+            // only "true" value is allowed
+            if (!instanceProfile.equalsIgnoreCase("true")) {
+                throw new CompilationException(INVALID_PARAM_VALUE_ALLOWED_VALUE, INSTANCE_PROFILE_FIELD_NAME, "true");
+            }
+
+            // no other authentication parameters are allowed
+            if (accessKeyId != null) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+                        INSTANCE_PROFILE_FIELD_NAME);
+            }
+            if (secretAccessKey != null) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
+                        INSTANCE_PROFILE_FIELD_NAME);
+            }
+            if (sessionToken != null) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, SESSION_TOKEN_FIELD_NAME,
+                        INSTANCE_PROFILE_FIELD_NAME);
+            }
+            credentialsProvider = InstanceProfileCredentialsProvider.create();
+        } else if (accessKeyId != null || secretAccessKey != null) {
+            // accessKeyId authentication
+            if (accessKeyId == null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+                        SECRET_ACCESS_KEY_FIELD_NAME);
+            }
+            if (secretAccessKey == null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
+                        ACCESS_KEY_ID_FIELD_NAME);
+            }
+
+            // use session token if provided
+            if (sessionToken != null) {
+                credentialsProvider = StaticCredentialsProvider
+                        .create(AwsSessionCredentials.create(accessKeyId, secretAccessKey, sessionToken));
+            } else {
+                credentialsProvider =
+                        StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKeyId, secretAccessKey));
+            }
+        } else {
+            // if only session token is provided, accessKeyId is required
+            throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+                    SESSION_TOKEN_FIELD_NAME);
+        }
+
+        builder.credentialsProvider(credentialsProvider);
+
+        // Validate the region
+        List<Region> regions = S3Client.serviceMetadata().regions();
+        Optional<Region> selectedRegion = regions.stream().filter(region -> region.id().equals(regionId)).findFirst();
+
+        if (selectedRegion.isEmpty()) {
+            throw new CompilationException(S3_REGION_NOT_SUPPORTED, regionId);
+        }
+        builder.region(selectedRegion.get());
+
+        // Validate the service endpoint if present
+        if (serviceEndpoint != null) {
+            try {
+                URI uri = new URI(serviceEndpoint);
+                try {
+                    builder.endpointOverride(uri);
+                } catch (NullPointerException ex) {
+                    throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+                }
+            } catch (URISyntaxException ex) {
+                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR,
+                        String.format("Invalid service endpoint %s", serviceEndpoint));
+            }
+        }
+
+        return builder.build();
+    }
+
+    /**
+     * Builds the S3 client using the provided configuration
+     *
+     * @param configuration      properties
+     * @param numberOfPartitions number of partitions in the cluster
+     */
+    public static void configureAwsS3HdfsJobConf(JobConf conf, Map<String, String> configuration,
+            int numberOfPartitions) {
+        String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+        String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+        String sessionToken = configuration.get(SESSION_TOKEN_FIELD_NAME);
+        String serviceEndpoint = configuration.get(SERVICE_END_POINT_FIELD_NAME);
+
+        //Disable caching S3 FileSystem
+        HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_S3_PROTOCOL);
+
+        /*
+         * Authentication Methods:
+         * 1- Anonymous: no accessKeyId and no secretAccessKey
+         * 2- Temporary: has to provide accessKeyId, secretAccessKey and sessionToken
+         * 3- Private: has to provide accessKeyId and secretAccessKey
+         */
+        if (accessKeyId == null) {
+            //Tells hadoop-aws it is an anonymous access
+            conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_ANONYMOUS_ACCESS);
+        } else {
+            conf.set(HADOOP_ACCESS_KEY_ID, accessKeyId);
+            conf.set(HADOOP_SECRET_ACCESS_KEY, secretAccessKey);
+            if (sessionToken != null) {
+                conf.set(HADOOP_SESSION_TOKEN, sessionToken);
+                //Tells hadoop-aws it is a temporary access
+                conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_TEMP_ACCESS);
+            }
+        }
+
+        /*
+         * This is to allow S3 definition to have path-style form. Should always be true to match the current
+         * way we access files in S3
+         */
+        conf.set(HADOOP_PATH_STYLE_ACCESS, ExternalDataConstants.TRUE);
+
+        /*
+         * Set the size of S3 connection pool to be the number of partitions
+         */
+        conf.set(HADOOP_S3_CONNECTION_POOL_SIZE, String.valueOf(numberOfPartitions));
+
+        if (serviceEndpoint != null) {
+            // Validation of the URL should be done at hadoop-aws level
+            conf.set(HADOOP_SERVICE_END_POINT, serviceEndpoint);
+        } else {
+            //Region is ignored and buckets could be found by the central endpoint
+            conf.set(HADOOP_SERVICE_END_POINT, Constants.CENTRAL_ENDPOINT);
+        }
+    }
+
+    /**
+     * Validate external dataset properties
+     *
+     * @param configuration properties
+     * @throws CompilationException Compilation exception
+     */
+    public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
+            IWarningCollector collector) throws CompilationException {
+
+        // check if the format property is present
+        if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+            throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+        }
+
+        // Both parameters should be passed, or neither should be passed (for anonymous/no auth)
+        String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+        String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+        if (accessKeyId == null || secretAccessKey == null) {
+            // If one is passed, the other is required
+            if (accessKeyId != null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
+                        ACCESS_KEY_ID_FIELD_NAME);
+            } else if (secretAccessKey != null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+                        SECRET_ACCESS_KEY_FIELD_NAME);
+            }
+        }
+
+        validateIncludeExclude(configuration);
+
+        // Check if the bucket is present
+        S3Client s3Client = buildAwsS3Client(configuration);
+        S3Response response;
+        boolean useOldApi = false;
+        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+        String prefix = getPrefix(configuration);
+
+        try {
+            response = isBucketEmpty(s3Client, container, prefix, false);
+        } catch (S3Exception ex) {
+            // Method not implemented, try falling back to old API
+            try {
+                // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
+                if (ex.awsErrorDetails().errorCode().equals(ERROR_METHOD_NOT_IMPLEMENTED)) {
+                    useOldApi = true;
+                    response = isBucketEmpty(s3Client, container, prefix, true);
+                } else {
+                    throw ex;
+                }
+            } catch (SdkException ex2) {
+                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+            }
+        } catch (SdkException ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        } finally {
+            if (s3Client != null) {
+                CleanupUtils.close(s3Client, null);
+            }
+        }
+
+        boolean isEmpty = useOldApi ? ((ListObjectsResponse) response).contents().isEmpty()
+                : ((ListObjectsV2Response) response).contents().isEmpty();
+        if (isEmpty && collector.shouldWarn()) {
+            Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+            collector.warn(warning);
+        }
+
+        // Returns 200 only in case the bucket exists, otherwise, throws an exception. However, to
+        // ensure coverage, check if the result is successful as well and not only catch exceptions
+        if (!response.sdkHttpResponse().isSuccessful()) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_CONTAINER_NOT_FOUND, container);
+        }
+    }
+
+    /**
+     * Checks for a single object in the specified bucket to determine if the bucket is empty or not.
+     *
+     * @param s3Client  s3 client
+     * @param container the container name
+     * @param prefix    Prefix to be used
+     * @param useOldApi flag whether to use the old API or not
+     * @return returns the S3 response
+     */
+    private static S3Response isBucketEmpty(S3Client s3Client, String container, String prefix, boolean useOldApi) {
+        S3Response response;
+        if (useOldApi) {
+            ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder();
+            listObjectsBuilder.prefix(prefix);
+            response = s3Client.listObjects(listObjectsBuilder.bucket(container).maxKeys(1).build());
+        } else {
+            ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder();
+            listObjectsBuilder.prefix(prefix);
+            response = s3Client.listObjectsV2(listObjectsBuilder.bucket(container).maxKeys(1).build());
+        }
+        return response;
+    }
+
+    /**
+     * Returns the lists of S3 objects.
+     *
+     * @param configuration         properties
+     * @param includeExcludeMatcher include/exclude matchers to apply
+     */
+    public static List<S3Object> listS3Objects(Map<String, String> configuration,
+            AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher,
+            IWarningCollector warningCollector) throws CompilationException {
+        // Prepare to retrieve the objects
+        List<S3Object> filesOnly;
+        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+        S3Client s3Client = buildAwsS3Client(configuration);
+        String prefix = getPrefix(configuration);
+
+        try {
+            filesOnly = listS3Objects(s3Client, container, prefix, includeExcludeMatcher);
+        } catch (S3Exception ex) {
+            // New API is not implemented, try falling back to old API
+            try {
+                // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
+                if (ex.awsErrorDetails().errorCode().equals(ERROR_METHOD_NOT_IMPLEMENTED)) {
+                    filesOnly = oldApiListS3Objects(s3Client, container, prefix, includeExcludeMatcher);
+                } else {
+                    throw ex;
+                }
+            } catch (SdkException ex2) {
+                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+            }
+        } catch (SdkException ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        } finally {
+            if (s3Client != null) {
+                CleanupUtils.close(s3Client, null);
+            }
+        }
+
+        // Warn if no files are returned
+        if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+            Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+            warningCollector.warn(warning);
+        }
+
+        return filesOnly;
+    }
+
+    /**
+     * Uses the latest API to retrieve the objects from the storage.
+     *
+     * @param s3Client              S3 client
+     * @param container             container name
+     * @param prefix                definition prefix
+     * @param includeExcludeMatcher include/exclude matchers to apply
+     */
+    private static List<S3Object> listS3Objects(S3Client s3Client, String container, String prefix,
+            AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher) {
+        String newMarker = null;
+        List<S3Object> filesOnly = new ArrayList<>();
+
+        ListObjectsV2Response listObjectsResponse;
+        ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder().bucket(container);
+        listObjectsBuilder.prefix(prefix);
+
+        while (true) {
+            // List the objects from the start, or from the last marker in case of truncated result
+            if (newMarker == null) {
+                listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.build());
+            } else {
+                listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.continuationToken(newMarker).build());
+            }
+
+            // Collect the paths to files only
+            collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
+                    includeExcludeMatcher.getMatchersList(), filesOnly);
+
+            // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
+            if (!listObjectsResponse.isTruncated()) {
+                break;
+            } else {
+                newMarker = listObjectsResponse.nextContinuationToken();
+            }
+        }
+
+        return filesOnly;
+    }
+
+    /**
+     * Uses the old API (in case the new API is not implemented) to retrieve the objects from the storage
+     *
+     * @param s3Client              S3 client
+     * @param container             container name
+     * @param prefix                definition prefix
+     * @param includeExcludeMatcher include/exclude matchers to apply
+     */
+    private static List<S3Object> oldApiListS3Objects(S3Client s3Client, String container, String prefix,
+            AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher) {
+        String newMarker = null;
+        List<S3Object> filesOnly = new ArrayList<>();
+
+        ListObjectsResponse listObjectsResponse;
+        ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder().bucket(container);
+        listObjectsBuilder.prefix(prefix);
+
+        while (true) {
+            // List the objects from the start, or from the last marker in case of truncated result
+            if (newMarker == null) {
+                listObjectsResponse = s3Client.listObjects(listObjectsBuilder.build());
+            } else {
+                listObjectsResponse = s3Client.listObjects(listObjectsBuilder.marker(newMarker).build());
+            }
+
+            // Collect the paths to files only
+            collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
+                    includeExcludeMatcher.getMatchersList(), filesOnly);
+
+            // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
+            if (!listObjectsResponse.isTruncated()) {
+                break;
+            } else {
+                newMarker = listObjectsResponse.nextMarker();
+            }
+        }
+
+        return filesOnly;
+    }
+
+    /**
+     * AWS S3 returns all the objects as paths, not differentiating between folder and files. The path is considered
+     * a file if it does not end up with a "/" which is the separator in a folder structure.
+     *
+     * @param s3Objects List of returned objects
+     */
+    private static void collectAndFilterFiles(List<S3Object> s3Objects, BiPredicate<List<Matcher>, String> predicate,
+            List<Matcher> matchers, List<S3Object> filesOnly) {
+        for (S3Object object : s3Objects) {
+            // skip folders
+            if (object.key().endsWith("/")) {
+                continue;
+            }
+
+            // No filter, add file
+            if (predicate.test(matchers, object.key())) {
+                filesOnly.add(object);
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureConstants.java
new file mode 100644
index 0000000..9ade27b
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureConstants.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.external.util.azure.blob_storage;
+
+/*
+ * Note: Azure Blob and Azure Datalake use identical authentication, so they are using the same properties.
+ * If they end up diverging, then properties for AzureBlob and AzureDataLake need to be created.
+ */
+public class AzureConstants {
+    private AzureConstants() {
+        throw new AssertionError("do not instantiate");
+    }
+
+    /*
+     * Asterix Configuration Keys
+     */
+    public static final String MANAGED_IDENTITY_ID_FIELD_NAME = "managedIdentityId";
+    public static final String ACCOUNT_NAME_FIELD_NAME = "accountName";
+    public static final String ACCOUNT_KEY_FIELD_NAME = "accountKey";
+    public static final String SHARED_ACCESS_SIGNATURE_FIELD_NAME = "sharedAccessSignature";
+    public static final String TENANT_ID_FIELD_NAME = "tenantId";
+    public static final String CLIENT_ID_FIELD_NAME = "clientId";
+    public static final String CLIENT_SECRET_FIELD_NAME = "clientSecret";
+    public static final String CLIENT_CERTIFICATE_FIELD_NAME = "clientCertificate";
+    public static final String CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME = "clientCertificatePassword";
+    public static final String ENDPOINT_FIELD_NAME = "endpoint";
+
+    // Specific Azure data lake property
+    /*
+    The behavior of Data Lake (true file system) is to read the files of the specified prefix only, example:
+    storage/myData/personal/file1.json
+    storage/myData/personal/file2.json
+    storage/myData/file3.json
+    If the prefix used is "myData", then only the file file3.json is read. However, if the property "recursive"
+    is set to "true" when creating the external dataset, then it goes recursively overall the paths, and the result
+    is file1.json, file2.json and file3.json.
+     */
+    public static final String RECURSIVE_FIELD_NAME = "recursive";
+
+    /*
+     * Hadoop-Azure
+     */
+    //Used when accountName and accessKey are provided
+    public static final String HADOOP_AZURE_FS_ACCOUNT_KEY = "fs.azure.account.key";
+    //Used when a connectionString is provided
+    public static final String HADOOP_AZURE_FS_SAS = "fs.azure.sas";
+    public static final String HADOOP_AZURE_BLOB_PROTOCOL = "wasbs";
+    public static final String HADOOP_AZURE_DATALAKE_PROTOCOL = "abfss";
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureUtils.java
new file mode 100644
index 0000000..0dc9ad2
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureUtils.java
@@ -0,0 +1,636 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.util.azure.blob_storage;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_SOURCE_ERROR;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_NOT_ALLOWED_AT_SAME_TIME;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_REQUIRED;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.external.util.ExternalDataUtils.getFirstNotNull;
+import static org.apache.asterix.external.util.ExternalDataUtils.getPrefix;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.ACCOUNT_KEY_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.ACCOUNT_NAME_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.CLIENT_CERTIFICATE_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.CLIENT_ID_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.CLIENT_SECRET_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.ENDPOINT_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_BLOB_PROTOCOL;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_FS_ACCOUNT_KEY;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_FS_SAS;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.MANAGED_IDENTITY_ID_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.RECURSIVE_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.SHARED_ACCESS_SIGNATURE_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.TENANT_ID_FIELD_NAME;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiPredicate;
+import java.util.regex.Matcher;
+
+import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+
+import com.azure.core.credential.AzureSasCredential;
+import com.azure.core.http.rest.PagedIterable;
+import com.azure.identity.ClientCertificateCredentialBuilder;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.BlobServiceClientBuilder;
+import com.azure.storage.blob.models.BlobItem;
+import com.azure.storage.blob.models.ListBlobsOptions;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import com.azure.storage.common.policy.RequestRetryOptions;
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.DataLakeServiceClientBuilder;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import com.azure.storage.file.datalake.models.PathItem;
+
+public class AzureUtils {
+    private AzureUtils() {
+        throw new AssertionError("do not instantiate");
+    }
+
+    /**
+     * Builds the Azure storage account using the provided configuration
+     *
+     * @param configuration properties
+     * @return client
+     */
+    public static BlobServiceClient buildAzureBlobClient(IApplicationContext appCtx, Map<String, String> configuration)
+            throws CompilationException {
+        String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
+        String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
+        String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
+        String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+        String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
+        String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
+        String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
+        String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
+        String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
+        String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
+
+        // Client builder
+        BlobServiceClientBuilder builder = new BlobServiceClientBuilder();
+        int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
+        RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
+        builder.retryOptions(requestRetryOptions);
+
+        // Endpoint is required
+        if (endpoint == null) {
+            throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
+        }
+        builder.endpoint(endpoint);
+
+        // Shared Key
+        if (accountName != null || accountKey != null) {
+            if (accountName == null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
+                        ACCOUNT_KEY_FIELD_NAME);
+            }
+
+            if (accountKey == null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
+                        ACCOUNT_NAME_FIELD_NAME);
+            }
+
+            Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
+                    MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+                    CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+            if (provided.isPresent()) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+                        ACCOUNT_KEY_FIELD_NAME);
+            }
+            StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
+            builder.credential(credential);
+        }
+
+        // Shared access signature
+        if (sharedAccessSignature != null) {
+            Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
+                    CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
+                    CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+            if (provided.isPresent()) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+                        SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+            }
+            AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
+            builder.credential(credential);
+        }
+
+        // Managed Identity auth
+        if (managedIdentityId != null) {
+            Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+                    CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+            if (provided.isPresent()) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+                        MANAGED_IDENTITY_ID_FIELD_NAME);
+            }
+            builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
+        }
+
+        // Client secret & certificate auth
+        if (clientId != null) {
+            // Both (or neither) client secret and client secret were provided, only one is allowed
+            if ((clientSecret == null) == (clientCertificate == null)) {
+                if (clientSecret != null) {
+                    throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
+                            CLIENT_CERTIFICATE_FIELD_NAME);
+                } else {
+                    throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
+                            CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
+                }
+            }
+
+            // Tenant ID is required
+            if (tenantId == null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
+                        CLIENT_ID_FIELD_NAME);
+            }
+
+            // Client certificate password is not allowed if client secret is used
+            if (clientCertificatePassword != null && clientSecret != null) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
+                        CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
+            }
+
+            // Use AD authentication
+            if (clientSecret != null) {
+                ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
+                secret.clientId(clientId);
+                secret.tenantId(tenantId);
+                secret.clientSecret(clientSecret);
+                builder.credential(secret.build());
+            } else {
+                // Certificate
+                ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
+                certificate.clientId(clientId);
+                certificate.tenantId(tenantId);
+                try {
+                    InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
+                    if (clientCertificatePassword == null) {
+                        Method pemCertificate = ClientCertificateCredentialBuilder.class
+                                .getDeclaredMethod("pemCertificate", InputStream.class);
+                        pemCertificate.setAccessible(true);
+                        pemCertificate.invoke(certificate, certificateContent);
+                    } else {
+                        Method pemCertificate = ClientCertificateCredentialBuilder.class
+                                .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
+                        pemCertificate.setAccessible(true);
+                        pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
+                    }
+                } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
+                    throw new CompilationException(EXTERNAL_SOURCE_ERROR, ex.getMessage());
+                }
+                builder.credential(certificate.build());
+            }
+        }
+
+        // If client id is not present, ensure client secret, certificate, tenant id and client certificate
+        // password are not present
+        if (clientId == null) {
+            Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
+                    CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+            if (provided.isPresent()) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+                        SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+            }
+        }
+
+        try {
+            return builder.buildClient();
+        } catch (Exception ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        }
+    }
+
+    /**
+     * Builds the Azure data lake storage account using the provided configuration
+     *
+     * @param configuration properties
+     * @return client
+     */
+    public static DataLakeServiceClient buildAzureDatalakeClient(IApplicationContext appCtx,
+            Map<String, String> configuration) throws CompilationException {
+        String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
+        String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
+        String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
+        String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+        String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
+        String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
+        String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
+        String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
+        String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
+        String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
+
+        // Client builder
+        DataLakeServiceClientBuilder builder = new DataLakeServiceClientBuilder();
+        int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
+        RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
+        builder.retryOptions(requestRetryOptions);
+
+        // Endpoint is required
+        if (endpoint == null) {
+            throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
+        }
+        builder.endpoint(endpoint);
+
+        // Shared Key
+        if (accountName != null || accountKey != null) {
+            if (accountName == null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
+                        ACCOUNT_KEY_FIELD_NAME);
+            }
+
+            if (accountKey == null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
+                        ACCOUNT_NAME_FIELD_NAME);
+            }
+
+            Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
+                    MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+                    CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+            if (provided.isPresent()) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+                        ACCOUNT_KEY_FIELD_NAME);
+            }
+            StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
+            builder.credential(credential);
+        }
+
+        // Shared access signature
+        if (sharedAccessSignature != null) {
+            Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
+                    CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
+                    CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+            if (provided.isPresent()) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+                        SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+            }
+            AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
+            builder.credential(credential);
+        }
+
+        // Managed Identity auth
+        if (managedIdentityId != null) {
+            Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+                    CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+            if (provided.isPresent()) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+                        MANAGED_IDENTITY_ID_FIELD_NAME);
+            }
+            builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
+        }
+
+        // Client secret & certificate auth
+        if (clientId != null) {
+            // Both (or neither) client secret and client secret were provided, only one is allowed
+            if ((clientSecret == null) == (clientCertificate == null)) {
+                if (clientSecret != null) {
+                    throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
+                            CLIENT_CERTIFICATE_FIELD_NAME);
+                } else {
+                    throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
+                            CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
+                }
+            }
+
+            // Tenant ID is required
+            if (tenantId == null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
+                        CLIENT_ID_FIELD_NAME);
+            }
+
+            // Client certificate password is not allowed if client secret is used
+            if (clientCertificatePassword != null && clientSecret != null) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
+                        CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
+            }
+
+            // Use AD authentication
+            if (clientSecret != null) {
+                ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
+                secret.clientId(clientId);
+                secret.tenantId(tenantId);
+                secret.clientSecret(clientSecret);
+                builder.credential(secret.build());
+            } else {
+                // Certificate
+                ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
+                certificate.clientId(clientId);
+                certificate.tenantId(tenantId);
+                try {
+                    InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
+                    if (clientCertificatePassword == null) {
+                        Method pemCertificate = ClientCertificateCredentialBuilder.class
+                                .getDeclaredMethod("pemCertificate", InputStream.class);
+                        pemCertificate.setAccessible(true);
+                        pemCertificate.invoke(certificate, certificateContent);
+                    } else {
+                        Method pemCertificate = ClientCertificateCredentialBuilder.class
+                                .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
+                        pemCertificate.setAccessible(true);
+                        pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
+                    }
+                } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
+                    throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+                }
+                builder.credential(certificate.build());
+            }
+        }
+
+        // If client id is not present, ensure client secret, certificate, tenant id and client certificate
+        // password are not present
+        if (clientId == null) {
+            Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
+                    CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+            if (provided.isPresent()) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+                        SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+            }
+        }
+
+        try {
+            return builder.buildClient();
+        } catch (Exception ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        }
+    }
+
+    public static List<BlobItem> listBlobItems(BlobServiceClient blobServiceClient, Map<String, String> configuration,
+            AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher,
+            IWarningCollector warningCollector) throws CompilationException {
+        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+        List<BlobItem> filesOnly = new ArrayList<>();
+
+        // Ensure the validity of include/exclude
+        validateIncludeExclude(configuration);
+
+        BlobContainerClient blobContainer;
+        try {
+            blobContainer = blobServiceClient.getBlobContainerClient(container);
+
+            // Get all objects in a container and extract the paths to files
+            ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
+            listBlobsOptions.setPrefix(getPrefix(configuration));
+            Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
+
+            // Collect the paths to files only
+            collectAndFilterBlobFiles(blobItems, includeExcludeMatcher.getPredicate(),
+                    includeExcludeMatcher.getMatchersList(), filesOnly);
+
+            // Warn if no files are returned
+            if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+                Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+                warningCollector.warn(warning);
+            }
+        } catch (Exception ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        }
+
+        return filesOnly;
+    }
+
+    /**
+     * Collects and filters the files only, and excludes any folders
+     *
+     * @param items     storage items
+     * @param predicate predicate to test with for file filtration
+     * @param matchers  include/exclude matchers to test against
+     * @param filesOnly List containing the files only (excluding folders)
+     */
+    private static void collectAndFilterBlobFiles(Iterable<BlobItem> items,
+            BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<BlobItem> filesOnly) {
+        for (BlobItem item : items) {
+            String uri = item.getName();
+
+            // skip folders
+            if (uri.endsWith("/")) {
+                continue;
+            }
+
+            // No filter, add file
+            if (predicate.test(matchers, uri)) {
+                filesOnly.add(item);
+            }
+        }
+    }
+
+    public static List<PathItem> listDatalakePathItems(DataLakeServiceClient client, Map<String, String> configuration,
+            AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher,
+            IWarningCollector warningCollector) throws CompilationException {
+        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+        List<PathItem> filesOnly = new ArrayList<>();
+
+        // Ensure the validity of include/exclude
+        validateIncludeExclude(configuration);
+
+        DataLakeFileSystemClient fileSystemClient;
+        try {
+            fileSystemClient = client.getFileSystemClient(container);
+
+            // Get all objects in a container and extract the paths to files
+            ListPathsOptions listOptions = new ListPathsOptions();
+            boolean recursive = Boolean.parseBoolean(configuration.get(RECURSIVE_FIELD_NAME));
+            listOptions.setRecursive(recursive);
+            listOptions.setPath(getPrefix(configuration, false));
+            PagedIterable<PathItem> pathItems = fileSystemClient.listPaths(listOptions, null);
+
+            // Collect the paths to files only
+            collectAndFilterDatalakeFiles(pathItems, includeExcludeMatcher.getPredicate(),
+                    includeExcludeMatcher.getMatchersList(), filesOnly);
+
+            // Warn if no files are returned
+            if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+                Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+                warningCollector.warn(warning);
+            }
+        } catch (Exception ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        }
+
+        return filesOnly;
+    }
+
+    /**
+     * Collects and filters the files only, and excludes any folders
+     *
+     * @param items     storage items
+     * @param predicate predicate to test with for file filtration
+     * @param matchers  include/exclude matchers to test against
+     * @param filesOnly List containing the files only (excluding folders)
+     */
+    private static void collectAndFilterDatalakeFiles(Iterable<PathItem> items,
+            BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<PathItem> filesOnly) {
+        for (PathItem item : items) {
+            String uri = item.getName();
+
+            // skip folders
+            if (uri.endsWith("/")) {
+                continue;
+            }
+
+            // No filter, add file
+            if (predicate.test(matchers, uri)) {
+                filesOnly.add(item);
+            }
+        }
+    }
+
+    /**
+     * Validate external dataset properties
+     *
+     * @param configuration properties
+     * @throws CompilationException Compilation exception
+     */
+    public static void validateAzureBlobProperties(Map<String, String> configuration, SourceLocation srcLoc,
+            IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
+
+        // check if the format property is present
+        if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+            throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+        }
+
+        validateIncludeExclude(configuration);
+
+        // Check if the bucket is present
+        BlobServiceClient blobServiceClient;
+        try {
+            String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+            blobServiceClient = buildAzureBlobClient(appCtx, configuration);
+            BlobContainerClient blobContainer = blobServiceClient.getBlobContainerClient(container);
+
+            // Get all objects in a container and extract the paths to files
+            ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
+            listBlobsOptions.setPrefix(getPrefix(configuration));
+            Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
+
+            if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
+                Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+                collector.warn(warning);
+            }
+        } catch (CompilationException ex) {
+            throw ex;
+        } catch (Exception ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        }
+    }
+
+    /**
+     * Validate external dataset properties
+     *
+     * @param configuration properties
+     * @throws CompilationException Compilation exception
+     */
+    public static void validateAzureDataLakeProperties(Map<String, String> configuration, SourceLocation srcLoc,
+            IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
+
+        // check if the format property is present
+        if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+            throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+        }
+
+        validateIncludeExclude(configuration);
+
+        // Check if the bucket is present
+        DataLakeServiceClient dataLakeServiceClient;
+        try {
+            String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+            dataLakeServiceClient = buildAzureDatalakeClient(appCtx, configuration);
+            DataLakeFileSystemClient fileSystemClient = dataLakeServiceClient.getFileSystemClient(container);
+
+            // Get all objects in a container and extract the paths to files
+            ListPathsOptions listPathsOptions = new ListPathsOptions();
+            listPathsOptions.setPath(getPrefix(configuration));
+            Iterable<PathItem> blobItems = fileSystemClient.listPaths(listPathsOptions, null);
+
+            if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
+                Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+                collector.warn(warning);
+            }
+        } catch (CompilationException ex) {
+            throw ex;
+        } catch (Exception ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        }
+    }
+
+    /**
+     * Builds the Azure Blob storage client using the provided configuration
+     *
+     * @param configuration properties
+     * @see <a href="https://docs.microsoft.com/en-us/azure/databricks/data/data-sources/azure/azure-storage">Azure
+     * Blob storage</a>
+     */
+    public static void configureAzureHdfsJobConf(JobConf conf, Map<String, String> configuration, String endPoint) {
+        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+        String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
+        String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+
+        //Disable caching S3 FileSystem
+        HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_AZURE_BLOB_PROTOCOL);
+
+        //Key for Hadoop configuration
+        StringBuilder hadoopKey = new StringBuilder();
+        //Value for Hadoop configuration
+        String hadoopValue;
+        if (accountKey != null || sharedAccessSignature != null) {
+            if (accountKey != null) {
+                hadoopKey.append(HADOOP_AZURE_FS_ACCOUNT_KEY).append('.');
+                //Set only the AccountKey
+                hadoopValue = accountKey;
+            } else {
+                //Use SAS for Hadoop FS as connectionString is provided
+                hadoopKey.append(HADOOP_AZURE_FS_SAS).append('.');
+                //Setting the container is required for SAS
+                hadoopKey.append(container).append('.');
+                //Set the connection string for SAS
+                hadoopValue = sharedAccessSignature;
+            }
+            //Set the endPoint, which includes the AccountName
+            hadoopKey.append(endPoint);
+            //Tells Hadoop we are reading from Blob Storage
+            conf.set(hadoopKey.toString(), hadoopValue);
+        }
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.java
new file mode 100644
index 0000000..6bf2266
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.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.external.util.google.gcs;
+
+public class GCSConstants {
+    private GCSConstants() {
+        throw new AssertionError("do not instantiate");
+    }
+
+    public static final String JSON_CREDENTIALS_FIELD_NAME = "jsonCredentials";
+    public static final String ENDPOINT_FIELD_NAME = "endpoint";
+
+    /*
+     * Hadoop internal configuration
+     */
+    public static final String HADOOP_GCS_PROTOCOL = "gs";
+
+    // hadoop credentials
+    public static final String HADOOP_AUTH_TYPE = "fs.gs.auth.type";
+    public static final String HADOOP_AUTH_UNAUTHENTICATED = "UNAUTHENTICATED";
+    public static final String HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE = "SERVICE_ACCOUNT_JSON_KEYFILE";
+    public static final String HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE_PATH =
+            "google.cloud.auth.service.account.json.keyfile";
+
+    // gs hadoop parameters
+    public static final String HADOOP_SUPPORT_COMPRESSED = "fs.gs.inputstream.support.gzip.encoding.enable";
+    public static final String HADOOP_ENDPOINT = "fs.gs.storage.root.url";
+    public static final String HADOOP_MAX_REQUESTS_PER_BATCH = "fs.gs.max.requests.per.batch";
+    public static final String HADOOP_BATCH_THREADS = "fs.gs.batch.threads";
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java
new file mode 100644
index 0000000..93dc272
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.util.google.gcs;
+
+import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_SOURCE_ERROR;
+import static org.apache.asterix.external.util.ExternalDataUtils.getPrefix;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.ENDPOINT_FIELD_NAME;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE_PATH;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_AUTH_TYPE;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_AUTH_UNAUTHENTICATED;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_ENDPOINT;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_GCS_PROTOCOL;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.JSON_CREDENTIALS_FIELD_NAME;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiPredicate;
+import java.util.regex.Matcher;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+
+import com.google.api.gax.paging.Page;
+import com.google.auth.oauth2.ServiceAccountCredentials;
+import com.google.cloud.BaseServiceException;
+import com.google.cloud.storage.Blob;
+import com.google.cloud.storage.Storage;
+import com.google.cloud.storage.StorageOptions;
+
+public class GCSUtils {
+    private GCSUtils() {
+        throw new AssertionError("do not instantiate");
+
+    }
+
+    /**
+     * Builds the client using the provided configuration
+     *
+     * @param configuration properties
+     * @return clientasterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+     * @throws CompilationException CompilationException
+     */
+    public static Storage buildClient(Map<String, String> configuration) throws CompilationException {
+        String jsonCredentials = configuration.get(JSON_CREDENTIALS_FIELD_NAME);
+        String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
+
+        StorageOptions.Builder builder = StorageOptions.newBuilder();
+
+        // Use credentials if available
+        if (jsonCredentials != null) {
+            try (InputStream credentialsStream = new ByteArrayInputStream(jsonCredentials.getBytes())) {
+                builder.setCredentials(ServiceAccountCredentials.fromStream(credentialsStream));
+            } catch (IOException ex) {
+                throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+            }
+        }
+
+        if (endpoint != null) {
+            builder.setHost(endpoint);
+        }
+
+        return builder.build().getService();
+    }
+
+    /**
+     * Validate external dataset properties
+     *
+     * @param configuration properties
+     * @throws CompilationException Compilation exception
+     */
+    public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
+            IWarningCollector collector) throws CompilationException {
+
+        // check if the format property is present
+        if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+            throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+        }
+
+        validateIncludeExclude(configuration);
+        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+        try {
+            Storage.BlobListOption limitOption = Storage.BlobListOption.pageSize(1);
+            Storage.BlobListOption prefixOption = Storage.BlobListOption.prefix(getPrefix(configuration));
+            Storage storage = buildClient(configuration);
+            Page<Blob> items = storage.list(container, limitOption, prefixOption);
+
+            if (!items.iterateAll().iterator().hasNext() && collector.shouldWarn()) {
+                Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+                collector.warn(warning);
+            }
+        } catch (CompilationException ex) {
+            throw ex;
+        } catch (Exception ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        }
+    }
+
+    public static List<Blob> listItems(Map<String, String> configuration, IncludeExcludeMatcher includeExcludeMatcher,
+            IWarningCollector warningCollector) throws CompilationException {
+        // Prepare to retrieve the objects
+        List<Blob> filesOnly = new ArrayList<>();
+        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+        Storage gcs = buildClient(configuration);
+        Storage.BlobListOption options = Storage.BlobListOption.prefix(ExternalDataUtils.getPrefix(configuration));
+        Page<Blob> items;
+
+        try {
+            items = gcs.list(container, options);
+        } catch (BaseServiceException ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        }
+
+        // Collect the paths to files only
+        collectAndFilterFiles(items, includeExcludeMatcher.getPredicate(), includeExcludeMatcher.getMatchersList(),
+                filesOnly);
+
+        // Warn if no files are returned
+        if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+            Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+            warningCollector.warn(warning);
+        }
+
+        return filesOnly;
+    }
+
+    /**
+     * Excludes paths ending with "/" as that's a directory indicator, we need to return the files only
+     *
+     * @param items List of returned objects
+     */
+    private static void collectAndFilterFiles(Page<Blob> items, BiPredicate<List<Matcher>, String> predicate,
+            List<Matcher> matchers, List<Blob> filesOnly) {
+        for (Blob item : items.iterateAll()) {
+            // skip folders
+            if (item.getName().endsWith("/")) {
+                continue;
+            }
+
+            // No filter, add file
+            if (predicate.test(matchers, item.getName())) {
+                filesOnly.add(item);
+            }
+        }
+    }
+
+    /**
+     * Builds the client using the provided configuration
+     *
+     * @param configuration      properties
+     * @param numberOfPartitions number of partitions in the cluster
+     */
+    public static void configureHdfsJobConf(JobConf conf, Map<String, String> configuration, int numberOfPartitions) {
+        String jsonCredentials = configuration.get(JSON_CREDENTIALS_FIELD_NAME);
+        String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
+
+        // disable caching FileSystem
+        HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_GCS_PROTOCOL);
+
+        // TODO(htowaileb): needs further testing, recommended to disable by gcs-hadoop team
+        conf.set(GCSConstants.HADOOP_SUPPORT_COMPRESSED, ExternalDataConstants.FALSE);
+
+        // TODO(htowaileb): needs further testing
+        // set number of threads
+        //        conf.set(GCSConstants.HADOOP_MAX_REQUESTS_PER_BATCH, String.valueOf(numberOfPartitions));
+        //        conf.set(GCSConstants.HADOOP_BATCH_THREADS, String.valueOf(numberOfPartitions));
+
+        // authentication method
+        // TODO(htowaileb): find a way to pass the content instead of the path to keyfile, this line is temporary
+        Path credentials = Path.of("credentials.json");
+        if (jsonCredentials == null) {
+            // anonymous access
+            conf.set(HADOOP_AUTH_TYPE, HADOOP_AUTH_UNAUTHENTICATED);
+        } else {
+            // TODO(htowaileb) need to pass the file content
+            conf.set(HADOOP_AUTH_TYPE, HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE);
+            conf.set(HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE_PATH, credentials.toAbsolutePath().toString());
+        }
+
+        // set endpoint if provided, default is https://storage.googleapis.com/
+        if (endpoint != null) {
+            conf.set(HADOOP_ENDPOINT, endpoint);
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-external-data/src/main/java/org/checkerframework/checker/nullness/compatqual/NullableDecl.java
similarity index 61%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-external-data/src/main/java/org/checkerframework/checker/nullness/compatqual/NullableDecl.java
index 6095b26..a4cb401 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-external-data/src/main/java/org/checkerframework/checker/nullness/compatqual/NullableDecl.java
@@ -16,9 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.checkerframework.checker.nullness.compatqual;
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
 
-select value v from range(1,2) v where v > ?;
+/*
+ * This is a clean room implementation of the NullableDecl interface based
+ * on 3 requirements:
+ * 1. shall be an @interface named NullableDecl
+ * 2. shall be in the org.checkerframework.checker.nullness.compatqual package
+ * 3. the rention policy for the interface shall be RUNTIME
+ */
+@Retention(RetentionPolicy.RUNTIME)
+public @interface NullableDecl {
+}
diff --git a/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
index dceed82..88f3fcb 100644
--- a/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
+++ b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
@@ -26,4 +26,5 @@
 org.apache.asterix.external.input.record.reader.azure.blob.AzureBlobReaderFactory
 org.apache.asterix.external.input.record.reader.azure.datalake.AzureDataLakeReaderFactory
 org.apache.asterix.external.input.record.reader.azure.parquet.AzureBlobParquetReaderFactory
-org.apache.asterix.external.input.record.reader.azure.parquet.AzureDataLakeParquetReaderFactory
\ No newline at end of file
+org.apache.asterix.external.input.record.reader.azure.parquet.AzureDataLakeParquetReaderFactory
+org.apache.asterix.external.input.record.reader.gcs.parquet.GCSParquetReaderFactory
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/AwsS3Test.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/AwsS3Test.java
index 90ea04b..91afbd8 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/AwsS3Test.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/AwsS3Test.java
@@ -18,8 +18,8 @@
  */
 package org.apache.asterix.external.input.record.reader.awss3;
 
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ERROR_INTERNAL_ERROR;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ERROR_SLOW_DOWN;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.ERROR_INTERNAL_ERROR;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.ERROR_SLOW_DOWN;
 
 import java.lang.reflect.Field;
 import java.lang.reflect.Method;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExtensionClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExtensionClause.java
new file mode 100644
index 0000000..af9015b
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExtensionClause.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.lang.common.base;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+
+public abstract class AbstractExtensionClause extends AbstractClause {
+    /**
+     * Parent languages should handle extension functionality via a new method in {@link IVisitorExtension}.
+     */
+    @Override
+    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, getSourceLocation(),
+                "Unhandled dispatch to an extension clause node!");
+    }
+
+    public abstract IVisitorExtension getVisitorExtension();
+
+    @Override
+    public final ClauseType getClauseType() {
+        return ClauseType.EXTENSION;
+    }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Clause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Clause.java
index 14f836b..95ed6df 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Clause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Clause.java
@@ -43,7 +43,9 @@
         SELECT_ELEMENT,
         SELECT_REGULAR,
         SELECT_SET_OPERATION,
-        UNNEST_CLAUSE
+        UNNEST_CLAUSE,
+
+        EXTENSION
     }
 
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IVisitorExtension.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IVisitorExtension.java
new file mode 100644
index 0000000..9007eb8
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IVisitorExtension.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.lang.common.base;
+
+import java.util.Collection;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.expression.AbstractCallExpression;
+import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.parser.ScopeChecker;
+import org.apache.asterix.lang.common.rewrites.VariableSubstitutionEnvironment;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+
+/**
+ * Contract for all extension AST nodes to allow existing rewrites to support language extensions.
+ */
+public interface IVisitorExtension {
+    Expression simpleExpressionDispatch(ILangVisitor<Expression, ILangExpression> simpleExpressionVisitor,
+            ILangExpression argument) throws CompilationException;
+
+    Void freeVariableDispatch(ILangVisitor<Void, Collection<VariableExpr>> freeVariableVisitor,
+            Collection<VariableExpr> freeVariables) throws CompilationException;
+
+    Void bindingVariableDispatch(ILangVisitor<Void, Collection<VariableExpr>> bindingVariableVisitor,
+            Collection<VariableExpr> bindingVariables) throws CompilationException;
+
+    Expression variableScopeDispatch(ILangVisitor<Expression, ILangExpression> scopingVisitor, ILangExpression argument,
+            ScopeChecker scopeChecker) throws CompilationException;
+
+    ILangExpression deepCopyDispatch(ILangVisitor<ILangExpression, Void> deepCopyVisitor) throws CompilationException;
+
+    Pair<ILangExpression, VariableSubstitutionEnvironment> remapCloneDispatch(
+            ILangVisitor<Pair<ILangExpression, VariableSubstitutionEnvironment>, VariableSubstitutionEnvironment> remapCloneVisitor,
+            VariableSubstitutionEnvironment substitutionEnvironment) throws CompilationException;
+
+    Boolean inlineUDFsDispatch(ILangVisitor<Boolean, Void> inlineUDFsVisitor) throws CompilationException;
+
+    Void gatherFunctionsDispatch(ILangVisitor<Void, Void> gatherFunctionsVisitor,
+            Collection<? super AbstractCallExpression> functionCalls) throws CompilationException;
+
+    Boolean checkSubqueryDispatch(ILangVisitor<Boolean, ILangExpression> checkSubqueryVisitor, ILangExpression argument)
+            throws CompilationException;
+
+    Boolean check92AggregateDispatch(ILangVisitor<Boolean, ILangExpression> check92AggregateVisitor,
+            ILangExpression argument) throws CompilationException;
+
+    Boolean checkNonFunctionalDispatch(ILangVisitor<Boolean, Void> checkNonFunctionalVisitor)
+            throws CompilationException;
+
+    Boolean checkDatasetOnlyDispatch(ILangVisitor<Boolean, VariableExpr> checkDatasetOnlyVisitor,
+            VariableExpr datasetCandidate) throws CompilationException;
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
index 2594cdd..4330c4e 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
@@ -107,6 +107,8 @@
         CREATE_SYNONYM,
         SYNONYM_DROP,
         VIEW_DROP,
+        ANALYZE,
+        ANALYZE_DROP,
         COMPACT,
         EXTERNAL_DATASET_REFRESH,
         SUBSCRIBE_FEED,
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 95cccb0..19b3cfa 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
@@ -32,7 +32,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.Counter;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 
-public final class LangRewritingContext {
+public class LangRewritingContext {
     private final MetadataProvider metadataProvider;
     private final IWarningCollector warningCollector;
     private final Map<FunctionSignature, FunctionDecl> declaredFunctions;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/AnalyzeDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/AnalyzeDropStatement.java
new file mode 100644
index 0000000..67cfeec
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/AnalyzeDropStatement.java
@@ -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.
+ */
+package org.apache.asterix.lang.common.statement;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+
+public class AnalyzeDropStatement extends AbstractStatement {
+
+    private final DataverseName dataverseName;
+    private final String datasetName;
+
+    public AnalyzeDropStatement(DataverseName dataverseName, String datasetName) {
+        this.dataverseName = dataverseName;
+        this.datasetName = datasetName;
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.ANALYZE_DROP;
+    }
+
+    public DataverseName getDataverseName() {
+        return dataverseName;
+    }
+
+    public String getDatasetName() {
+        return datasetName;
+    }
+
+    @Override
+    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+        return visitor.visit(this, arg);
+    }
+
+    @Override
+    public byte getCategory() {
+        return Category.DDL;
+    }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/AnalyzeStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/AnalyzeStatement.java
new file mode 100644
index 0000000..cbf2c07
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/AnalyzeStatement.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.statement;
+
+import java.util.Locale;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.lang.common.util.ExpressionUtils;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.asterix.object.base.AdmBigIntNode;
+import org.apache.asterix.object.base.AdmDoubleNode;
+import org.apache.asterix.object.base.AdmObjectNode;
+import org.apache.asterix.object.base.AdmStringNode;
+import org.apache.asterix.object.base.IAdmNode;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+
+public class AnalyzeStatement extends AbstractStatement {
+
+    private static final String SAMPLE_FIELD_NAME = "sample";
+    private static final String SAMPLE_LOW = "low";
+    private static final String SAMPLE_MEDIUM = "medium";
+    private static final String SAMPLE_HIGH = "high";
+    private static final int SAMPLE_LOW_SIZE = 1063;
+    private static final int SAMPLE_MEDIUM_SIZE = SAMPLE_LOW_SIZE * 4;
+    private static final int SAMPLE_HIGH_SIZE = SAMPLE_MEDIUM_SIZE * 4;
+    private static final int SAMPLE_DEFAULT_SIZE = SAMPLE_LOW_SIZE;
+
+    private static final String SAMPLE_SEED_FIELD_NAME = "sample-seed";
+
+    private final DataverseName dataverseName;
+    private final String datasetName;
+    private final AdmObjectNode options;
+
+    public AnalyzeStatement(DataverseName dataverseName, String datasetName, RecordConstructor options)
+            throws CompilationException {
+        this.dataverseName = dataverseName;
+        this.datasetName = datasetName;
+        this.options = options == null ? null : validateOptions(ExpressionUtils.toNode(options));
+    }
+
+    private static AdmObjectNode validateOptions(AdmObjectNode options) throws CompilationException {
+        for (String fieldName : options.getFieldNames()) {
+            switch (fieldName) {
+                case SAMPLE_FIELD_NAME:
+                case SAMPLE_SEED_FIELD_NAME:
+                    break;
+                default:
+                    throw new CompilationException(ErrorCode.INVALID_PARAM, fieldName);
+            }
+        }
+        return options;
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.ANALYZE;
+    }
+
+    public DataverseName getDataverseName() {
+        return dataverseName;
+    }
+
+    public String getDatasetName() {
+        return datasetName;
+    }
+
+    public int getSampleSize() throws CompilationException {
+        IAdmNode n = getOption(SAMPLE_FIELD_NAME);
+        if (n == null) {
+            return SAMPLE_DEFAULT_SIZE;
+        }
+        switch (n.getType()) {
+            case STRING:
+                String s = ((AdmStringNode) n).get();
+                switch (s.toLowerCase(Locale.ROOT)) {
+                    case SAMPLE_LOW:
+                        return SAMPLE_LOW_SIZE;
+                    case SAMPLE_MEDIUM:
+                        return SAMPLE_MEDIUM_SIZE;
+                    case SAMPLE_HIGH:
+                        return SAMPLE_HIGH_SIZE;
+                    default:
+                        throw new CompilationException(ErrorCode.INVALID_PROPERTY_FORMAT, SAMPLE_FIELD_NAME);
+                }
+            case BIGINT:
+                int v = (int) ((AdmBigIntNode) n).get();
+                if (!isValidSampleSize(v)) {
+                    throw new CompilationException(ErrorCode.INVALID_PROPERTY_FORMAT, SAMPLE_FIELD_NAME);
+                }
+                return v;
+            case DOUBLE:
+                v = (int) ((AdmDoubleNode) n).get();
+                if (!isValidSampleSize(v)) {
+                    throw new CompilationException(ErrorCode.INVALID_PROPERTY_FORMAT, SAMPLE_FIELD_NAME);
+                }
+                return v;
+            default:
+                throw new CompilationException(ErrorCode.WITH_FIELD_MUST_BE_OF_TYPE, SAMPLE_FIELD_NAME,
+                        BuiltinType.ASTRING.getTypeName(), n.getType().toString());
+        }
+    }
+
+    public long getOrCreateSampleSeed() throws AlgebricksException {
+        IAdmNode n = getOption(SAMPLE_SEED_FIELD_NAME);
+        return n != null ? getSampleSeed(n) : createSampleSeed();
+    }
+
+    private long getSampleSeed(IAdmNode n) throws CompilationException {
+        switch (n.getType()) {
+            case BIGINT:
+                return ((AdmBigIntNode) n).get();
+            case DOUBLE:
+                return (long) ((AdmDoubleNode) n).get();
+            case STRING:
+                String s = ((AdmStringNode) n).get();
+                try {
+                    return Long.parseLong(s);
+                } catch (NumberFormatException e) {
+                    throw new CompilationException(ErrorCode.INVALID_PROPERTY_FORMAT, SAMPLE_SEED_FIELD_NAME);
+                }
+            default:
+                throw new CompilationException(ErrorCode.WITH_FIELD_MUST_BE_OF_TYPE, SAMPLE_SEED_FIELD_NAME,
+                        BuiltinType.AINT64.getTypeName(), n.getType().toString());
+        }
+    }
+
+    private long createSampleSeed() {
+        return System.nanoTime() + System.identityHashCode(this);
+    }
+
+    private boolean isValidSampleSize(int v) {
+        return v >= SAMPLE_LOW_SIZE && v <= SAMPLE_HIGH_SIZE;
+    }
+
+    private IAdmNode getOption(String optionName) {
+        return options != null ? options.get(optionName) : null;
+    }
+
+    @Override
+    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+        return visitor.visit(this, arg);
+    }
+
+    @Override
+    public byte getCategory() {
+        return Category.DDL;
+    }
+}
\ No newline at end of file
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 c3583698..d479626 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
@@ -27,10 +27,10 @@
 
 public class IndexDropStatement extends AbstractStatement {
 
-    private DataverseName dataverseName;
-    private Identifier datasetName;
-    private Identifier indexName;
-    private boolean ifExists;
+    private final DataverseName dataverseName;
+    private final Identifier datasetName;
+    private final Identifier indexName;
+    private final boolean ifExists;
 
     public IndexDropStatement(DataverseName dataverseName, Identifier datasetName, Identifier indexName,
             boolean ifExists) {
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 52775d3..077748d 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
@@ -31,6 +31,7 @@
 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.IVisitorExtension;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.LimitClause;
@@ -264,6 +265,11 @@
     }
 
     @Override
+    public Boolean visit(IVisitorExtension ve, Void arg) throws CompilationException {
+        return ve.inlineUDFsDispatch(this);
+    }
+
+    @Override
     public Boolean visit(InsertStatement insert, Void arg) throws CompilationException {
         boolean changed = false;
         Expression returnExpression = insert.getReturnExpression();
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 d73c264..06f22b7 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
@@ -29,6 +29,7 @@
 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.IVisitorExtension;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.LimitClause;
@@ -339,6 +340,12 @@
     }
 
     @Override
+    public Pair<ILangExpression, VariableSubstitutionEnvironment> visit(IVisitorExtension ve,
+            VariableSubstitutionEnvironment arg) throws CompilationException {
+        return ve.remapCloneDispatch(this, arg);
+    }
+
+    @Override
     public Pair<ILangExpression, VariableSubstitutionEnvironment> visit(FieldAccessor fa,
             VariableSubstitutionEnvironment env) throws CompilationException {
         Pair<ILangExpression, VariableSubstitutionEnvironment> p = fa.getExpr().accept(this, env);
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
index 0ddbeb4..f42f2f0 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
@@ -65,6 +65,8 @@
 import org.apache.asterix.lang.common.expression.UnorderedListTypeDefinition;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.statement.AdapterDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeStatement;
 import org.apache.asterix.lang.common.statement.CompactStatement;
 import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
@@ -929,6 +931,23 @@
     }
 
     @Override
+    public Void visit(AnalyzeStatement as, Integer step) throws CompilationException {
+        out.print(skip(step) + "analyze dataset ");
+        out.print(generateFullName(as.getDataverseName(), as.getDatasetName()));
+        out.println(SEMICOLON);
+        return null;
+    }
+
+    @Override
+    public Void visit(AnalyzeDropStatement as, Integer step) throws CompilationException {
+        out.print(skip(step) + "analyze dataset ");
+        out.print(generateFullName(as.getDataverseName(), as.getDatasetName()));
+        out.print(" drop statistics");
+        out.println(SEMICOLON);
+        return null;
+    }
+
+    @Override
     public Void visit(CompactStatement del, Integer step) throws CompilationException {
         return null;
     }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
index b7cf7af..6dcfb83 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
@@ -25,6 +25,7 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.LimitClause;
@@ -227,6 +228,11 @@
     }
 
     @Override
+    public Void visit(IVisitorExtension ve, Void arg) throws CompilationException {
+        return ve.gatherFunctionsDispatch(this, calls);
+    }
+
+    @Override
     public Void visit(WhereClause wc, Void arg) throws CompilationException {
         wc.getWhereExpr().accept(this, arg);
         return null;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractAstVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractAstVisitor.java
index a422ef1..a6092c6 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractAstVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractAstVisitor.java
@@ -19,6 +19,8 @@
 package org.apache.asterix.lang.common.visitor.base;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.LimitClause;
@@ -130,4 +132,8 @@
         return null;
     }
 
+    @Override
+    public R visit(IVisitorExtension ve, T arg) throws CompilationException {
+        throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, "Extension dispatch not implemented!");
+    }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
index 91fe664..a060d1e 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
@@ -25,6 +25,8 @@
 import org.apache.asterix.lang.common.expression.TypeReferenceExpression;
 import org.apache.asterix.lang.common.expression.UnorderedListTypeDefinition;
 import org.apache.asterix.lang.common.statement.AdapterDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeStatement;
 import org.apache.asterix.lang.common.statement.CompactStatement;
 import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
@@ -261,6 +263,16 @@
     }
 
     @Override
+    public R visit(AnalyzeStatement as, T arg) throws CompilationException {
+        return null;
+    }
+
+    @Override
+    public R visit(AnalyzeDropStatement as, T arg) throws CompilationException {
+        return null;
+    }
+
+    @Override
     public R visit(CompactStatement del, T arg) throws CompilationException {
         return null;
     }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
index c2b1311..8fda66e 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.lang.common.visitor.base;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.LimitClause;
@@ -42,6 +43,8 @@
 import org.apache.asterix.lang.common.expression.UnorderedListTypeDefinition;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.statement.AdapterDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeStatement;
 import org.apache.asterix.lang.common.statement.CompactStatement;
 import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
@@ -207,6 +210,10 @@
 
     R visit(SynonymDropStatement del, T arg) throws CompilationException;
 
+    R visit(AnalyzeStatement as, T arg) throws CompilationException;
+
+    R visit(AnalyzeDropStatement as, T step) throws CompilationException;
+
     R visit(CompactStatement del, T arg) throws CompilationException;
 
     R visit(ListSliceExpression expression, T arg) throws CompilationException;
@@ -216,4 +223,6 @@
     R visit(ViewDropStatement vds, T arg) throws CompilationException;
 
     R visit(ViewDecl vd, T arg) throws CompilationException;
+
+    R visit(IVisitorExtension ve, T arg) throws CompilationException;
 }
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 ea6bac9..c902dbe 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
@@ -19,23 +19,43 @@
 
 package org.apache.asterix.lang.sqlpp.clause;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 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 extends AbstractClause {
-
+    private final List<List<String>> fieldExclusions = new ArrayList<>();
     private SelectElement selectElement;
     private SelectRegular selectRegular;
     private boolean distinct;
 
-    public SelectClause(SelectElement selectElement, SelectRegular selectRegular, boolean distinct) {
+    public SelectClause(SelectElement selectElement, SelectRegular selectRegular, List<List<String>> fieldExclusions,
+            boolean distinct) {
+        if (selectElement != null && selectRegular != null) {
+            throw new IllegalArgumentException("SELECT-ELEMENT and SELECT-REGULAR cannot both be specified.");
+        }
+        if (selectElement != null && fieldExclusions != null && !fieldExclusions.isEmpty()) {
+            throw new IllegalArgumentException("SELECT-ELEMENT and EXCLUDE cannot both be specified.");
+        }
+
         this.selectElement = selectElement;
         this.selectRegular = selectRegular;
         this.distinct = distinct;
+        if (fieldExclusions != null) {
+            this.fieldExclusions.addAll(fieldExclusions);
+        }
+    }
+
+    public SelectClause(SelectElement selectElement, SelectRegular selectRegular, boolean distinct) {
+        this(selectElement, selectRegular, null, distinct);
     }
 
     @Override
@@ -48,7 +68,12 @@
         return ClauseType.SELECT_CLAUSE;
     }
 
-    public void setSelectElement(SelectElement selectElement) {
+    public void setSelectElement(SelectElement selectElement) throws CompilationException {
+        if (!fieldExclusions.isEmpty() && selectElement != null) {
+            // We forbid SELECT VALUE and EXCLUDE at the parser, so we should never reach here.
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, getSourceLocation(),
+                    "SELECT ELEMENT and EXCLUDE cannot coexist!");
+        }
         this.selectElement = selectElement;
         this.selectRegular = null;
     }
@@ -82,15 +107,25 @@
         this.distinct = distinct;
     }
 
+    public List<List<String>> getFieldExclusions() {
+        return fieldExclusions;
+    }
+
     @Override
     public String toString() {
-        return "select " + (distinct ? "distinct " : "")
-                + (selectElement() ? "element " + selectElement : String.valueOf(selectRegular));
+        String distinctString = distinct ? "distinct " : "";
+        String valueString = selectElement() ? ("element " + selectElement) : String.valueOf(selectRegular);
+        String exceptString = "";
+        if (!fieldExclusions.isEmpty()) {
+            final Function<List<String>, String> fieldBuilder = f -> String.join(".", f);
+            exceptString = " exclude " + fieldExclusions.stream().map(fieldBuilder).collect(Collectors.joining(", "));
+        }
+        return String.format("select %s%s%s", distinctString, valueString, exceptString);
     }
 
     @Override
     public int hashCode() {
-        return Objects.hash(distinct, selectElement, selectRegular);
+        return Objects.hash(distinct, selectElement, selectRegular, fieldExclusions);
     }
 
     @Override
@@ -103,6 +138,7 @@
         }
         SelectClause target = (SelectClause) object;
         return distinct == target.distinct && Objects.equals(selectElement, target.selectElement)
-                && Objects.equals(selectRegular, target.selectRegular);
+                && Objects.equals(selectRegular, target.selectRegular)
+                && Objects.equals(fieldExclusions, target.fieldExclusions);
     }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppHint.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppHint.java
index 925140b..f565fcd 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppHint.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppHint.java
@@ -33,6 +33,7 @@
     DATETIME_ADD_RAND_HOURS_HINT("datetime-add-rand-hours"),
     DATETIME_BETWEEN_YEARS_HINT("datetime-between-years"),
     HASH_GROUP_BY_HINT("hash"),
+    HASH_JOIN_HINT("hashjoin"),
     INDEXED_NESTED_LOOP_JOIN_HINT("indexnl"),
     INMEMORY_HINT("inmem"),
     INSERT_RAND_INT_HINT("insert-rand-int"),
@@ -46,6 +47,8 @@
     VAL_FILE_HINT("val-files"),
     VAL_FILE_SAME_INDEX_HINT("val-file-same-idx"),
     GEN_FIELDS_HINT("gen-fields"),
+    SINGLE_DATASET_PREDICATE_SELECTIVITY_HINT("selectivity"),
+    JOIN_PREDICATE_PRODUCTIVITY_HINT("productivity"),
 
     // data generator hints
     DGEN_HINT("dgen");
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
index 557f17d..32549d9 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
@@ -61,7 +61,7 @@
 /**
  * This rewriter is used to rewrite body expression of user defined functions and views
  */
-class SqlppFunctionBodyRewriter extends SqlppQueryRewriter {
+public class SqlppFunctionBodyRewriter extends SqlppQueryRewriter {
 
     public SqlppFunctionBodyRewriter(IParserFactory parserFactory) {
         super(parserFactory);
@@ -97,6 +97,9 @@
         // Inlines column aliases.
         inlineColumnAlias();
 
+        // Rewrite SELECT EXCLUDE to use OBJECT_REMOVE_FIELDS.
+        rewriteSelectExcludeSugar();
+
         // Window expression core rewrites.
         rewriteWindowExpressions();
 
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 2f4fcc8..38177d8 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
@@ -58,6 +58,7 @@
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.InlineColumnAliasVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.InlineWithExpressionVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.OperatorExpressionVisitor;
+import org.apache.asterix.lang.sqlpp.rewrites.visitor.SelectExcludeRewriteSugarVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SetOperationVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlCompatRewriteVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppCaseAggregateExtractionVisitor;
@@ -162,6 +163,9 @@
         // Inlines column aliases.
         inlineColumnAlias();
 
+        // Rewrite SELECT EXCLUDE to use OBJECT_REMOVE_FIELDS.
+        rewriteSelectExcludeSugar();
+
         // Window expression core rewrites.
         rewriteWindowExpressions();
 
@@ -350,6 +354,12 @@
         }
     }
 
+    protected void rewriteSelectExcludeSugar() throws CompilationException {
+        SelectExcludeRewriteSugarVisitor selectExcludeRewriteSugarVisitor =
+                new SelectExcludeRewriteSugarVisitor(context);
+        rewriteTopExpr(selectExcludeRewriteSugarVisitor, null);
+    }
+
     private <R, T> R rewriteTopExpr(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
         R result = topStatement.accept(visitor, arg);
         logExpression(">>>> AST After", visitor.getClass().getSimpleName());
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 93a14ca..7967f36 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
@@ -45,6 +45,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.algebricks.core.algebra.expressions.PredicateCardinalityAnnotation;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class OperatorExpressionVisitor extends AbstractSqlppExpressionScopingVisitor {
@@ -85,6 +86,7 @@
     private Expression processLikeOperator(OperatorExpr operatorExpr, OperatorType opType) {
         CallExpr likeExpr =
                 new CallExpr(new FunctionSignature(BuiltinFunctions.STRING_LIKE), operatorExpr.getExprList());
+        likeExpr.addHints(operatorExpr.getHints());
         likeExpr.setSourceLocation(operatorExpr.getSourceLocation());
         switch (opType) {
             case LIKE:
@@ -135,6 +137,20 @@
         return callExpr;
     }
 
+    private List<IExpressionAnnotation> removeSelectivityHints(OperatorExpr expr) {
+        if (expr.hasHints()) {
+            List<IExpressionAnnotation> copyHintsExceptSelectivity = new ArrayList<>();
+            for (IExpressionAnnotation h : expr.getHints()) {
+                if (!(h.getClass().equals(PredicateCardinalityAnnotation.class))) {
+                    copyHintsExceptSelectivity.add(h);
+                }
+            }
+            return copyHintsExceptSelectivity;
+        } else {
+            return expr.getHints();
+        }
+    }
+
     private Expression processBetweenOperator(OperatorExpr operatorExpr, OperatorType opType)
             throws CompilationException {
         // The grammar guarantees that the BETWEEN operator gets exactly three expressions.
@@ -147,8 +163,11 @@
                 operatorExpr.getSourceLocation());
         // Creates the expression target <= right.
         Expression targetCopy = (Expression) SqlppRewriteUtil.deepCopy(target);
+
+        // remove any selectivity hints from operatorExpr; do not want to duplicate those hints
         Expression rightComparison = createOperatorExpression(OperatorType.LE, targetCopy, right,
-                operatorExpr.getHints(), operatorExpr.getSourceLocation());
+                removeSelectivityHints(operatorExpr), operatorExpr.getSourceLocation());
+
         Expression andExpr = createOperatorExpression(OperatorType.AND, leftComparison, rightComparison, null,
                 operatorExpr.getSourceLocation());
         switch (opType) {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SelectExcludeRewriteSugarVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SelectExcludeRewriteSugarVisitor.java
new file mode 100644
index 0000000..e900c03
--- /dev/null
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SelectExcludeRewriteSugarVisitor.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.lang.sqlpp.rewrites.visitor;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+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;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.context.Scope;
+import org.apache.asterix.lang.common.expression.CallExpr;
+import org.apache.asterix.lang.common.expression.ListConstructor;
+import org.apache.asterix.lang.common.expression.LiteralExpr;
+import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.literal.StringLiteral;
+import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
+import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.lang.sqlpp.clause.FromClause;
+import org.apache.asterix.lang.sqlpp.clause.FromTerm;
+import org.apache.asterix.lang.sqlpp.clause.Projection;
+import org.apache.asterix.lang.sqlpp.clause.SelectBlock;
+import org.apache.asterix.lang.sqlpp.clause.SelectClause;
+import org.apache.asterix.lang.sqlpp.clause.SelectElement;
+import org.apache.asterix.lang.sqlpp.clause.SelectRegular;
+import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
+import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.struct.SetOperationInput;
+import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
+import org.apache.asterix.lang.sqlpp.util.SqlppRewriteUtil;
+import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
+import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppExpressionScopingVisitor;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+/**
+ * Rewrites the exclusion list of a SELECT clause into a OBJECT_REMOVE_FIELDS function application. This rewrite
+ * <b>MUST</b> run after {@link InlineColumnAliasVisitor}.
+ * <p>
+ * Input:
+ * <pre>
+ *   FROM     ...
+ *   WHERE    ...
+ *   SELECT   * EXCLUDE a, b.f
+ *   ORDER BY c
+ * </pre>
+ * Output:
+ * <pre>
+ *   FROM     ( FROM   ...
+ *              WHERE  ...
+ *              SELECT *
+ *              ORDER BY c ) TMP_1
+ *   SELECT   VALUE OBJECT_REMOVE_FIELDS(TMP_1, [ "a", [ "b", "f" ]])
+ * </pre>
+ * <p>
+ * There exists a special case with a single {@link FromTerm} node (with no other local bindings) and a SELECT * clause,
+ * where we qualify our field exclusion list with the {@link FromTerm} variable if we cannot anchor on the dataset
+ * variable. For example:
+ * <pre>
+ *   FROM   MyDataset D
+ *   SELECT * EXCLUDE a, b.c, D.d
+ * </pre>
+ * Is conceptually processed as:
+ * <pre>
+ *   FROM   MyDataset D
+ *   SELECT * EXCLUDE D.a, D.b.c, D.d
+ * </pre>
+ * For all other cases, our EXCLUDE will work solely with what our SELECT returns.
+ */
+public class SelectExcludeRewriteSugarVisitor extends AbstractSqlppExpressionScopingVisitor {
+    public SelectExcludeRewriteSugarVisitor(LangRewritingContext langRewritingContext) {
+        super(langRewritingContext);
+    }
+
+    @Override
+    public Expression visit(SelectBlock selectBlock, ILangExpression arg) throws CompilationException {
+        super.visit(selectBlock, arg);
+
+        // Proceed if we have field-exclusions.
+        SelectClause selectClause = selectBlock.getSelectClause();
+        if (selectClause.getFieldExclusions().isEmpty()) {
+            return null;
+        }
+        SelectExpression selectExpression = (SelectExpression) arg;
+
+        // If we have a single dataset in a FROM-CLAUSE (with no other variables in our local scope / grouping)...
+        if (selectBlock.hasFromClause() && selectBlock.getFromClause().getFromTerms().size() == 1) {
+            FromTerm fromTerm = selectBlock.getFromClause().getFromTerms().get(0);
+            if (!selectBlock.hasGroupbyClause() && !fromTerm.hasCorrelateClauses() && selectBlock.getLetWhereList()
+                    .stream().noneMatch(c -> c.getClauseType() == Clause.ClauseType.LET_CLAUSE)) {
+                // ...and we have a 'SELECT *'...
+                SelectRegular selectRegular = selectClause.getSelectRegular();
+                if (selectClause.selectRegular() && selectRegular.getProjections().size() == 1
+                        && selectRegular.getProjections().get(0).getKind() == Projection.Kind.STAR) {
+                    // ...then qualify our field exclusions with our FROM-CLAUSE variable.
+                    String fromTermName = fromTerm.getLeftVariable().getVar().getValue();
+                    String qualifier = SqlppVariableUtil.toUserDefinedName(fromTermName);
+                    selectClause.getFieldExclusions().stream().filter(e -> {
+                        // Do not needlessly qualify names that are already bound to variables in our scope.
+                        // Note: We use our local scope to include the single-dataset variable AND our outer scope.
+                        //       We already know that there are no other variables in our local scope.
+                        Iterator<Pair<Identifier, Set<? extends Scope.SymbolAnnotation>>> liveSymbolIterator =
+                                scopeChecker.getCurrentScope().liveSymbols(null);
+                        while (liveSymbolIterator.hasNext()) {
+                            Pair<Identifier, Set<? extends Scope.SymbolAnnotation>> symbol = liveSymbolIterator.next();
+                            String symbolName = SqlppVariableUtil.toUserDefinedName(symbol.first.getValue());
+                            if (symbolName.equals(e.get(0))) {
+                                return false;
+                            }
+                        }
+                        return true;
+                    }).forEach(e -> e.add(0, qualifier));
+                }
+            }
+        }
+
+        // Find our parent SET-OP-INPUT.
+        SetOperationInput setOperationInput = null;
+        SelectSetOperation selectSetOperation = selectExpression.getSelectSetOperation();
+        if (selectBlock.equals(selectSetOperation.getLeftInput().getSelectBlock())) {
+            setOperationInput = selectSetOperation.getLeftInput();
+        } else {
+            for (SetOperationRight rightInput : selectSetOperation.getRightInputs()) {
+                SetOperationInput setOperationRightInput = rightInput.getSetOperationRightInput();
+                if (selectBlock.equals(setOperationRightInput.getSelectBlock())) {
+                    setOperationInput = setOperationRightInput;
+                    break;
+                }
+            }
+        }
+        if (setOperationInput == null) {
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, selectBlock.getSourceLocation(),
+                    "Parent SET-OP-INPUT not found while rewriting SELECT-EXCLUDE!");
+        }
+
+        // Nest our original SELECT-BLOCK.
+        SourceLocation sourceLocation = selectBlock.getSourceLocation();
+        SetOperationInput innerSetOpInput = new SetOperationInput(selectBlock, null);
+        SelectSetOperation innerSelectSetOp = new SelectSetOperation(innerSetOpInput, null);
+        innerSelectSetOp.setSourceLocation(sourceLocation);
+        SelectExpression innerSelectExpr;
+        if (!selectSetOperation.hasRightInputs()) {
+            // We need to attach our LET / ORDER BY / LIMIT to our inner SELECT-EXPR.
+            SelectExpression selectExprCopy = (SelectExpression) SqlppRewriteUtil.deepCopy(selectExpression);
+            innerSelectExpr = new SelectExpression(selectExprCopy.getLetList(), innerSelectSetOp,
+                    selectExprCopy.getOrderbyClause(), selectExprCopy.getLimitClause(), true);
+            selectExpression.getLetList().clear();
+            selectExpression.setOrderbyClause(null);
+            selectExpression.setLimitClause(null);
+        } else {
+            innerSelectExpr = new SelectExpression(null, innerSelectSetOp, null, null, true);
+        }
+        innerSelectExpr.setSourceLocation(sourceLocation);
+
+        // Build a new SELECT-BLOCK.
+        VarIdentifier fromTermVariable = context.newVariable();
+        VariableExpr fromTermVariableExpr = new VariableExpr(fromTermVariable);
+        SelectClause innerSelectClause = buildSelectClause(selectClause, fromTermVariable);
+        innerSelectClause.setSourceLocation(sourceLocation);
+        FromTerm innerFromTerm = new FromTerm(innerSelectExpr, fromTermVariableExpr, null, null);
+        innerFromTerm.setSourceLocation(sourceLocation);
+        FromClause innerFromClause = new FromClause(List.of(innerFromTerm));
+        innerFromClause.setSourceLocation(sourceLocation);
+        SelectBlock innerSelectBlock = new SelectBlock(innerSelectClause, innerFromClause, null, null, null);
+        setOperationInput.setSelectBlock(innerSelectBlock);
+        return null;
+    }
+
+    private SelectClause buildSelectClause(SelectClause originalSelectClause, VarIdentifier iterationVariable) {
+        // Convert our list of identifiers into a list of literals representing field names.
+        ListConstructor listConstructor = new ListConstructor();
+        listConstructor.setType(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR);
+        listConstructor.setExprList(new ArrayList<>());
+        for (List<String> nestedField : originalSelectClause.getFieldExclusions()) {
+            if (nestedField.size() == 1) {
+                // For non-nested fields, we do not wrap our name in a list.
+                listConstructor.getExprList().add(new LiteralExpr(new StringLiteral(nestedField.get(0))));
+            } else {
+                // Otherwise, build a list to insert into our list.
+                ListConstructor nestedFieldList = new ListConstructor();
+                nestedFieldList.setType(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR);
+                nestedFieldList.setExprList(nestedField.stream().map(f -> new LiteralExpr(new StringLiteral(f)))
+                        .collect(Collectors.toList()));
+                listConstructor.getExprList().add(nestedFieldList);
+            }
+        }
+        List<Expression> objectRemoveFieldsArguments = new ArrayList<>();
+        objectRemoveFieldsArguments.add(new VariableExpr(iterationVariable));
+        objectRemoveFieldsArguments.add(listConstructor);
+        originalSelectClause.getFieldExclusions().clear();
+
+        // Remove the DISTINCT from our original SELECT-CLAUSE, if it exists.
+        boolean isDistinct = originalSelectClause.distinct();
+        if (isDistinct) {
+            originalSelectClause.setDistinct(false);
+        }
+
+        // Create the call to OBJECT_REMOVE_FIELDS.
+        FunctionSignature functionSignature = new FunctionSignature(BuiltinFunctions.REMOVE_FIELDS);
+        CallExpr callExpr = new CallExpr(functionSignature, objectRemoveFieldsArguments);
+        SelectElement selectElement = new SelectElement(callExpr);
+        SelectClause selectClause = new SelectClause(selectElement, null, isDistinct);
+        selectClause.setSourceLocation(originalSelectClause.getSourceLocation());
+        return selectClause;
+    }
+}
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 d337de9..c4fb951 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
@@ -124,7 +124,8 @@
         return expr.accept(visitor, selectBlock);
     }
 
-    private List<Pair<Expression, Identifier>> createGroupFieldList(SelectBlock selectBlock) {
+    private List<Pair<Expression, Identifier>> createGroupFieldList(SelectBlock selectBlock)
+            throws CompilationException {
         List<Pair<Expression, Identifier>> groupFieldList = new ArrayList<>();
         addToFieldList(groupFieldList, SqlppVariableUtil.getBindingVariables(selectBlock.getFromClause()));
         addToFieldList(groupFieldList, SqlppVariableUtil.getLetBindingVariables(selectBlock.getLetWhereList()));
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
index bf3e227..aa548519 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
@@ -139,7 +139,8 @@
         winExpr.setExprList(newExprList);
     }
 
-    private List<Pair<Expression, Identifier>> createWindowFieldList(SelectBlock selectBlock) {
+    private List<Pair<Expression, Identifier>> createWindowFieldList(SelectBlock selectBlock)
+            throws CompilationException {
         List<Pair<Expression, Identifier>> fieldList = new ArrayList<>();
         if (selectBlock != null) {
             addToFieldList(fieldList, SqlppVariableUtil.getBindingVariables(selectBlock.getFromClause()));
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java
index 9d50160..0a8e528 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java
@@ -29,17 +29,11 @@
 import org.apache.asterix.lang.common.base.Clause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
-import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
-import org.apache.asterix.lang.common.expression.GbyVariableExpressionPair;
-import org.apache.asterix.lang.common.expression.QuantifiedExpression;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.struct.Identifier;
-import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
-import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
-import org.apache.asterix.lang.sqlpp.clause.FromClause;
-import org.apache.asterix.lang.sqlpp.clause.FromTerm;
+import org.apache.asterix.lang.sqlpp.visitor.BindingVariableVisitor;
 import org.apache.asterix.lang.sqlpp.visitor.FreeVariableVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -131,61 +125,14 @@
         return freeVars;
     }
 
-    public static List<VariableExpr> getBindingVariables(FromClause fromClause) {
-        if (fromClause == null) {
+    public static List<VariableExpr> getBindingVariables(ILangExpression langExpr) throws CompilationException {
+        if (langExpr == null) {
             return Collections.emptyList();
         }
-        List<VariableExpr> bindingVars = new ArrayList<>();
-        for (FromTerm fromTerm : fromClause.getFromTerms()) {
-            bindingVars.addAll(getBindingVariables(fromTerm));
-        }
-        return bindingVars;
-    }
 
-    public static List<VariableExpr> getBindingVariables(FromTerm fromTerm) {
+        final BindingVariableVisitor visitor = new BindingVariableVisitor();
         List<VariableExpr> bindingVars = new ArrayList<>();
-        if (fromTerm == null) {
-            return bindingVars;
-        }
-        bindingVars.add(fromTerm.getLeftVariable());
-        if (fromTerm.hasPositionalVariable()) {
-            bindingVars.add(fromTerm.getPositionalVariable());
-        }
-        for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
-            bindingVars.add(correlateClause.getRightVariable());
-            if (correlateClause.hasPositionalVariable()) {
-                bindingVars.add(correlateClause.getPositionalVariable());
-            }
-        }
-        return bindingVars;
-    }
-
-    public static List<VariableExpr> getBindingVariables(GroupbyClause gbyClause) {
-        List<VariableExpr> bindingVars = new ArrayList<>();
-        if (gbyClause == null) {
-            return bindingVars;
-        }
-        Set<VariableExpr> gbyKeyVars = new HashSet<>();
-        for (List<GbyVariableExpressionPair> gbyPairList : gbyClause.getGbyPairList()) {
-            for (GbyVariableExpressionPair gbyKey : gbyPairList) {
-                VariableExpr var = gbyKey.getVar();
-                if (var != null && gbyKeyVars.add(var)) {
-                    bindingVars.add(var);
-                }
-            }
-        }
-        if (gbyClause.hasDecorList()) {
-            for (GbyVariableExpressionPair gbyKey : gbyClause.getDecorPairList()) {
-                VariableExpr var = gbyKey.getVar();
-                if (var != null) {
-                    bindingVars.add(var);
-                }
-            }
-        }
-        if (gbyClause.hasWithMap()) {
-            bindingVars.addAll(gbyClause.getWithVarMap().values());
-        }
-        bindingVars.add(gbyClause.getGroupVar());
+        langExpr.accept(visitor, bindingVars);
         return bindingVars;
     }
 
@@ -203,15 +150,6 @@
         return bindingVars;
     }
 
-    public static List<VariableExpr> getBindingVariables(QuantifiedExpression qe) {
-        List<QuantifiedPair> quantifiedList = qe.getQuantifiedList();
-        List<VariableExpr> bindingVars = new ArrayList<>(quantifiedList.size());
-        for (QuantifiedPair qp : quantifiedList) {
-            bindingVars.add(qp.getVarExpr());
-        }
-        return bindingVars;
-    }
-
     public static void addToFieldVariableList(VariableExpr varExpr, List<Pair<Expression, Identifier>> outFieldList) {
         VarIdentifier var = varExpr.getVar();
         VariableExpr newVarExpr = new VariableExpr(var);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/BindingVariableVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/BindingVariableVisitor.java
new file mode 100644
index 0000000..1b7dc24
--- /dev/null
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/BindingVariableVisitor.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.asterix.lang.sqlpp.visitor;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
+import org.apache.asterix.lang.common.clause.GroupbyClause;
+import org.apache.asterix.lang.common.expression.GbyVariableExpressionPair;
+import org.apache.asterix.lang.common.expression.QuantifiedExpression;
+import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.struct.QuantifiedPair;
+import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
+import org.apache.asterix.lang.sqlpp.clause.FromClause;
+import org.apache.asterix.lang.sqlpp.clause.FromTerm;
+import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppAstVisitor;
+
+public class BindingVariableVisitor extends AbstractSqlppAstVisitor<Void, Collection<VariableExpr>> {
+    @Override
+    public Void visit(FromClause fromClause, Collection<VariableExpr> bindingVars) throws CompilationException {
+        for (FromTerm fromTerm : fromClause.getFromTerms()) {
+            fromTerm.accept(this, bindingVars);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(FromTerm fromTerm, Collection<VariableExpr> bindingVars) throws CompilationException {
+        bindingVars.add(fromTerm.getLeftVariable());
+        if (fromTerm.hasPositionalVariable()) {
+            bindingVars.add(fromTerm.getPositionalVariable());
+        }
+        for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
+            bindingVars.add(correlateClause.getRightVariable());
+            if (correlateClause.hasPositionalVariable()) {
+                bindingVars.add(correlateClause.getPositionalVariable());
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(GroupbyClause groupbyClause, Collection<VariableExpr> bindingVars) throws CompilationException {
+        Set<VariableExpr> gbyKeyVars = new HashSet<>();
+        for (List<GbyVariableExpressionPair> gbyPairList : groupbyClause.getGbyPairList()) {
+            for (GbyVariableExpressionPair gbyKey : gbyPairList) {
+                VariableExpr var = gbyKey.getVar();
+                if (var != null && gbyKeyVars.add(var)) {
+                    bindingVars.add(var);
+                }
+            }
+        }
+        if (groupbyClause.hasDecorList()) {
+            for (GbyVariableExpressionPair gbyKey : groupbyClause.getDecorPairList()) {
+                VariableExpr var = gbyKey.getVar();
+                if (var != null) {
+                    bindingVars.add(var);
+                }
+            }
+        }
+        if (groupbyClause.hasWithMap()) {
+            bindingVars.addAll(groupbyClause.getWithVarMap().values());
+        }
+        bindingVars.add(groupbyClause.getGroupVar());
+        return null;
+    }
+
+    @Override
+    public Void visit(QuantifiedExpression qe, Collection<VariableExpr> bindingVars) throws CompilationException {
+        List<QuantifiedPair> quantifiedList = qe.getQuantifiedList();
+        for (QuantifiedPair qp : quantifiedList) {
+            bindingVars.add(qp.getVarExpr());
+        }
+        return null;
+    }
+
+    @Override
+    public Void visit(IVisitorExtension ve, Collection<VariableExpr> arg) throws CompilationException {
+        return ve.bindingVariableDispatch(this, arg);
+    }
+}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckDatasetOnlyResolutionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckDatasetOnlyResolutionVisitor.java
index e4ccef5..f700d21 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckDatasetOnlyResolutionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckDatasetOnlyResolutionVisitor.java
@@ -21,6 +21,7 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.LimitClause;
@@ -160,6 +161,11 @@
     }
 
     @Override
+    public Boolean visit(IVisitorExtension ve, VariableExpr arg) throws CompilationException {
+        return ve.checkDatasetOnlyDispatch(this, arg);
+    }
+
+    @Override
     public Boolean visit(IfExpr ifexpr, VariableExpr arg) throws CompilationException {
         return false;
     }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckNonFunctionalExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckNonFunctionalExpressionVisitor.java
index d6ee1fc..d16322e 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckNonFunctionalExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckNonFunctionalExpressionVisitor.java
@@ -22,6 +22,7 @@
 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.IVisitorExtension;
 import org.apache.asterix.lang.common.expression.CallExpr;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppContainsExpressionVisitor;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -68,4 +69,9 @@
         }
         return super.visit(callExpr, arg);
     }
+
+    @Override
+    public Boolean visit(IVisitorExtension ve, Void arg) throws CompilationException {
+        return ve.checkNonFunctionalDispatch(this);
+    }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java
index 3192541..5e6a84a 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java
@@ -24,6 +24,7 @@
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.LimitClause;
@@ -157,6 +158,11 @@
     }
 
     @Override
+    public Boolean visit(IVisitorExtension ve, ILangExpression arg) throws CompilationException {
+        return ve.check92AggregateDispatch(this, arg);
+    }
+
+    @Override
     public Boolean visit(IfExpr ifexpr, ILangExpression parentSelectBlock) throws CompilationException {
         if (ifexpr.getCondExpr().accept(this, parentSelectBlock)) {
             return true;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSubqueryVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSubqueryVisitor.java
index a903279..399e463 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSubqueryVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSubqueryVisitor.java
@@ -24,6 +24,7 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.LimitClause;
@@ -231,6 +232,11 @@
     }
 
     @Override
+    public Boolean visit(IVisitorExtension ve, ILangExpression arg) throws CompilationException {
+        return ve.checkSubqueryDispatch(this, arg);
+    }
+
+    @Override
     public Boolean visit(IfExpr ifexpr, ILangExpression arg) throws CompilationException {
         return visit(ifexpr.getCondExpr(), arg) || visit(ifexpr.getThenExpr(), arg) || visit(ifexpr.getElseExpr(), arg);
     }
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 d74d047..1d43d0b 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
@@ -28,6 +28,7 @@
 import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.LimitClause;
@@ -193,7 +194,14 @@
         if (selectClause.selectRegular()) {
             selectRegular = (SelectRegular) selectClause.getSelectRegular().accept(this, arg);
         }
-        SelectClause copy = new SelectClause(selectElement, selectRegular, selectClause.distinct());
+        List<List<String>> fieldExclusions = new ArrayList<>();
+        if (!selectClause.getFieldExclusions().isEmpty()) {
+            for (List<String> fieldExclusion : selectClause.getFieldExclusions()) {
+                List<String> fieldExclusionCopy = new ArrayList<>(fieldExclusion);
+                fieldExclusions.add(fieldExclusionCopy);
+            }
+        }
+        SelectClause copy = new SelectClause(selectElement, selectRegular, fieldExclusions, selectClause.distinct());
         copy.setSourceLocation(selectClause.getSourceLocation());
         return copy;
     }
@@ -507,6 +515,11 @@
     }
 
     @Override
+    public ILangExpression visit(IVisitorExtension ve, Void arg) throws CompilationException {
+        return ve.deepCopyDispatch(this);
+    }
+
+    @Override
     public ILangExpression visit(CaseExpression caseExpr, Void arg) throws CompilationException {
         Expression conditionExpr = (Expression) caseExpr.getConditionExpr().accept(this, arg);
         List<Expression> whenExprList = copyExprList(caseExpr.getWhenExprs(), arg);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/FreeVariableVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/FreeVariableVisitor.java
index 9115b1c..8ab87e7 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/FreeVariableVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/FreeVariableVisitor.java
@@ -27,6 +27,7 @@
 import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Clause.ClauseType;
 import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.LimitClause;
@@ -468,6 +469,12 @@
     }
 
     @Override
+    public Void visit(IVisitorExtension ve, Collection<VariableExpr> arg) throws CompilationException {
+        ve.freeVariableDispatch(this, arg);
+        return null;
+    }
+
+    @Override
     public Void visit(CaseExpression caseExpr, Collection<VariableExpr> freeVars) throws CompilationException {
         caseExpr.getConditionExpr().accept(this, freeVars);
         visit(caseExpr.getWhenExprs(), freeVars);
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 c803d83..cab6e87 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
@@ -22,12 +22,14 @@
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
 import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
 import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
@@ -178,6 +180,11 @@
     public Void visit(SelectClause selectClause, Integer step) throws CompilationException {
         if (selectClause.selectRegular()) {
             selectClause.getSelectRegular().accept(this, step);
+            if (!selectClause.getFieldExclusions().isEmpty()) {
+                out.print(skip(step) + "EXCLUDE ");
+                out.println(selectClause.getFieldExclusions().stream().map(e -> String.join(".", e))
+                        .collect(Collectors.joining(",")));
+            }
         }
         if (selectClause.selectElement()) {
             selectClause.getSelectElement().accept(this, step);
@@ -373,6 +380,12 @@
     }
 
     @Override
+    public Void visit(IVisitorExtension ve, Integer arg) throws CompilationException {
+        // Language extensions should create a child of this class.
+        return null;
+    }
+
+    @Override
     public Void visit(WindowExpression winExpr, Integer step) throws CompilationException {
         out.print(skip(step) + "WINDOW ");
         printFunctionSignature(out, winExpr.getFunctionSignature(), winExpr.getFunctionSignature().getArity());
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 ce0d0a1..bff57e2 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
@@ -269,7 +269,15 @@
         } else {
             Pair<ILangExpression, VariableSubstitutionEnvironment> newSelectRegular =
                     selectClause.getSelectRegular().accept(this, env);
-            SelectClause newSelectClause = new SelectClause(null, (SelectRegular) newSelectRegular.first, distinct);
+            List<List<String>> fieldExclusions = new ArrayList<>();
+            if (!selectClause.getFieldExclusions().isEmpty()) {
+                for (List<String> fieldExclusion : selectClause.getFieldExclusions()) {
+                    List<String> fieldExclusionCopy = new ArrayList<>(fieldExclusion);
+                    fieldExclusions.add(fieldExclusionCopy);
+                }
+            }
+            SelectClause newSelectClause =
+                    new SelectClause(null, (SelectRegular) newSelectRegular.first, fieldExclusions, distinct);
             newSelectClause.setSourceLocation(selectClause.getSourceLocation());
             return new Pair<>(newSelectClause, newSelectRegular.second);
         }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java
index 975379d..4fa6bdc 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java
@@ -20,10 +20,12 @@
 
 import java.io.PrintWriter;
 import java.util.List;
+import java.util.stream.Collectors;
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.expression.GbyVariableExpressionPair;
@@ -171,6 +173,10 @@
     public Void visit(SelectClause selectClause, Integer step) throws CompilationException {
         if (selectClause.selectRegular()) {
             selectClause.getSelectRegular().accept(this, step);
+            if (!selectClause.getFieldExclusions().isEmpty()) {
+                out.println("exclude " + selectClause.getFieldExclusions().stream().map(e -> String.join(".", e))
+                        .collect(Collectors.joining(COMMA)));
+            }
         }
         if (selectClause.selectElement()) {
             selectClause.getSelectElement().accept(this, step);
@@ -284,6 +290,12 @@
     }
 
     @Override
+    public Void visit(IVisitorExtension ve, Integer arg) throws CompilationException {
+        // Language extensions should create a child of this class.
+        return null;
+    }
+
+    @Override
     public Void visit(InsertStatement insert, Integer step) throws CompilationException {
         out.print(skip(step) + "insert into " + datasetSymbol
                 + generateFullName(insert.getDataverseName(), insert.getDatasetName()) + "\n");
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppAstVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppAstVisitor.java
index e9c0e4c..7973841 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppAstVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppAstVisitor.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.lang.sqlpp.visitor.base;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
 import org.apache.asterix.lang.common.expression.ListSliceExpression;
 import org.apache.asterix.lang.common.visitor.base.AbstractAstVisitor;
 import org.apache.asterix.lang.sqlpp.clause.FromClause;
@@ -121,4 +122,9 @@
     public R visit(ListSliceExpression expression, T arg) throws CompilationException {
         return null;
     }
+
+    @Override
+    public R visit(IVisitorExtension ve, T arg) throws CompilationException {
+        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 7539046..765c2be 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
@@ -32,6 +32,7 @@
 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.base.IVisitorExtension;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.LimitClause;
@@ -413,6 +414,11 @@
         return winExpr;
     }
 
+    @Override
+    public Expression visit(IVisitorExtension ve, ILangExpression arg) throws CompilationException {
+        return ve.variableScopeDispatch(this, arg, scopeChecker);
+    }
+
     // Adds a new encountered alias identifier into a scope
     private void addNewVarSymbolToScope(Scope scope, VarIdentifier var, SourceLocation sourceLoc,
             SqlppVariableAnnotation... varAnnotations) throws CompilationException {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
index ef8b43c..6bbb740 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.IVisitorExtension;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.LimitClause;
@@ -384,6 +385,11 @@
     }
 
     @Override
+    public Expression visit(IVisitorExtension ve, ILangExpression arg) throws CompilationException {
+        return ve.simpleExpressionDispatch(this, arg);
+    }
+
+    @Override
     public Expression visit(CaseExpression caseExpr, ILangExpression arg) throws CompilationException {
         caseExpr.setConditionExpr(visit(caseExpr.getConditionExpr(), arg));
         caseExpr.setWhenExprs(visit(caseExpr.getWhenExprs(), arg));
@@ -403,7 +409,7 @@
         return null;
     }
 
-    protected Expression visit(Expression expr, ILangExpression arg) throws CompilationException {
+    public Expression visit(Expression expr, ILangExpression arg) throws CompilationException {
         return postVisit(preVisit(expr).accept(this, arg));
     }
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index 4e64944..64c73af 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -76,6 +76,7 @@
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
+import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
 import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.AbstractLangExpression;
 import org.apache.asterix.lang.common.base.AbstractStatement;
@@ -124,6 +125,8 @@
 import org.apache.asterix.lang.common.literal.TrueLiteral;
 import org.apache.asterix.lang.common.parser.ScopeChecker;
 import org.apache.asterix.lang.common.statement.AdapterDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeDropStatement;
 import org.apache.asterix.lang.common.statement.CompactStatement;
 import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
 import org.apache.asterix.lang.common.statement.StartFeedStatement;
@@ -205,7 +208,7 @@
 import org.apache.asterix.lang.sqlpp.util.ExpressionToVariableUtil;
 import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
 import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
-import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
+import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.asterix.om.exceptions.TypeMismatchException;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.BuiltinType;
@@ -215,7 +218,10 @@
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.HashJoinExpressionAnnotation;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.JoinProductivityAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.PredicateCardinalityAnnotation;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -223,6 +229,8 @@
 import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
 import org.apache.hyracks.util.LogRedactionUtil;
 import org.apache.hyracks.util.StringUtil;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 class SQLPPParser extends ScopeChecker implements IParser {
 
@@ -257,6 +265,7 @@
     private static final String REPLACE = "REPLACE";
     private static final String RETURNS = "RETURNS";
     private static final String CONFIG = "CONFIG";
+    private static final String STATISTICS = "STATISTICS";
 
 
     private static final String INT_TYPE_NAME = "int";
@@ -647,10 +656,45 @@
     private IExpressionAnnotation parseExpressionAnnotation(Token hintToken) {
       // placeholder for the annotation that should be returned if this hint's parameters cannot be parsed
       IExpressionAnnotation onParseErrorReturn = null;
+      double selectivity, cardinality, productivity;
+      Pattern number = Pattern.compile("\\d+\\.\\d+");
+      Pattern stringNumber = Pattern.compile("\\w+\\s+\\d+\\.\\d+");
+      Pattern lessThanOnePat = Pattern.compile("0\\.\\d+");
       try {
         switch (hintToken.hint) {
+          case SINGLE_DATASET_PREDICATE_SELECTIVITY_HINT:
+            selectivity = 1.0; // uninitialized
+            if (hintToken.hintParams != null) {
+            Matcher mat = lessThanOnePat.matcher(hintToken.hintParams);
+            if (mat.find()) {
+                selectivity = Double.parseDouble (mat.group());
+                }
+            }
+
+            return new PredicateCardinalityAnnotation(selectivity);
+          case JOIN_PREDICATE_PRODUCTIVITY_HINT:
+            productivity = 1.0; // uninitialized
+            String leftSideDataSet = null;
+            if (hintToken.hintParams != null) {
+                Matcher StringNum = stringNumber.matcher(hintToken.hintParams);
+
+                if (StringNum.find()) {
+                    String matchedGroup = StringNum.group();
+                    Pattern var = Pattern.compile("[a-zA-Z]\\w*"); // any word character [a-zA-Z_0-9]
+                    Matcher matVar = var.matcher(matchedGroup);
+                    if (matVar.find())
+                        leftSideDataSet = matVar.group();
+                    Matcher numMat = number.matcher(matchedGroup);
+                    if (numMat.find())
+                        productivity = Double.parseDouble (numMat.group());
+                }
+            }
+            // attach hint to global scope
+            return new JoinProductivityAnnotation (productivity, leftSideDataSet);
           case HASH_BROADCAST_JOIN_HINT:
             return new BroadcastExpressionAnnotation(BroadcastExpressionAnnotation.BroadcastSide.RIGHT);
+          case HASH_JOIN_HINT:
+            return new HashJoinExpressionAnnotation(HashJoinExpressionAnnotation.BuildSide.RIGHT);
           case INDEXED_NESTED_LOOP_JOIN_HINT:
             if (hintToken.hintParams == null) {
               return IndexedNLJoinExpressionAnnotation.INSTANCE_ANY_INDEX;
@@ -836,6 +880,7 @@
     | stmt = UpsertStatement()
     | stmt = ConnectionStatement()
     | stmt = CompactStatement()
+    | stmt = AnalyzeStatement()
     | stmt = Query()
     | stmt = RefreshExternalDatasetStatement()
   )
@@ -1378,7 +1423,7 @@
   <ON> nameComponents = QualifiedName() (<TYPE> <BTREE>)?
   {
     if (indexName == null) {
-      indexName = "primary_idx_" + nameComponents.second;
+      indexName = MetadataConstants.PRIMARY_INDEX_PREFIX + nameComponents.second;
     }
     CreateIndexStatement stmt = new CreateIndexStatement(nameComponents.first, nameComponents.second,
       new Identifier(indexName), IndexType.BTREE, Collections.emptyList(), false, -1, null, ifNotExists, null, null,
@@ -2628,6 +2673,50 @@
     }
 }
 
+Statement AnalyzeStatement() throws ParseException:
+{
+  Token startToken = null;
+  Statement stmt = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
+}
+{
+  <ANALYZE> { startToken = token; } Dataset() nameComponents = QualifiedName()
+  (
+    stmt = AnalyzeDatasetDropStatement(startToken, nameComponents.first, nameComponents.second)
+    | stmt = AnalyzeDatasetStatement(startToken, nameComponents.first, nameComponents.second)
+  )
+  {
+    return stmt;
+  }
+}
+
+Statement AnalyzeDatasetStatement(Token startToken, DataverseName dvName, Identifier identifier) throws ParseException:
+{
+  RecordConstructor withRecord = null;
+}
+{
+  ( <WITH> withRecord = RecordConstructor() )?
+  {
+    try {
+    AnalyzeStatement stmt = new AnalyzeStatement(dvName, identifier.getValue(), withRecord);
+    return addSourceLocation(stmt, startToken);
+    } catch (CompilationException e) {
+       throw new SqlppParseException(getSourceLocation(startToken), e.getMessage());
+    }
+  }
+}
+
+Statement AnalyzeDatasetDropStatement(Token startToken, DataverseName dvName, Identifier identifier) throws ParseException:
+{
+}
+{
+  <DROP> <IDENTIFIER> { expectToken(STATISTICS); }
+  {
+    AnalyzeDropStatement stmt = new AnalyzeDropStatement(dvName, identifier.getValue());
+    return addSourceLocation(stmt, startToken);
+  }
+}
+
 Statement CompactStatement() throws ParseException:
 {
   Token startToken = null;
@@ -2994,7 +3083,7 @@
   // Note: there's a copy of this production in PrimaryExpr() (LOOKAHEAD for FunctionCallExpr())
   //       that copy must be kept in sync with this code
   prefix = MultipartIdentifierWithHints(
-    SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT, SqlppHint.RANGE_HINT, SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT,
+    SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT, SqlppHint.RANGE_HINT, SqlppHint.HASH_JOIN_HINT, SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT,
     SqlppHint.SPATIAL_JOIN_HINT, SqlppHint.USE_SECONDARY_INDEX_SEARCH_HINT
   )
   (<SHARP> suffix = Identifier())?
@@ -3481,6 +3570,7 @@
   Token opToken = null;
   Expression operand = null;
   IExpressionAnnotation annotation = null;
+  List<IExpressionAnnotation> annotationList = new ArrayList<IExpressionAnnotation>();
 }
 {
     operand = BetweenExpr()
@@ -3494,10 +3584,13 @@
           }
           Token hintToken = fetchHint(token,
             SqlppHint.HASH_BROADCAST_JOIN_HINT, SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT,
-            SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT, SqlppHint.USE_SECONDARY_INDEX_SEARCH_HINT
+            SqlppHint.HASH_JOIN_HINT, SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT, SqlppHint.USE_SECONDARY_INDEX_SEARCH_HINT,
+            SqlppHint.SINGLE_DATASET_PREDICATE_SELECTIVITY_HINT, SqlppHint.JOIN_PREDICATE_PRODUCTIVITY_HINT
           );
-          if (hintToken != null) {
+          while (hintToken != null) {
             annotation = parseExpressionAnnotation(hintToken);
+            annotationList.add(annotation);
+            hintToken = hintToken.specialToken;
           }
           String operator = opToken.image.toLowerCase();
           if (operator.equals("<>")){
@@ -3527,7 +3620,7 @@
 
      {
        if (annotation != null) {
-         op.addHint(annotation);
+         op.addHints(annotationList);
        }
        return op==null? operand: op;
      }
@@ -3547,8 +3640,8 @@
       (<NOT> { not = true; })? <BETWEEN>
         {
           Token hintToken = fetchHint(token,
-            SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT, SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT,
-            SqlppHint.USE_SECONDARY_INDEX_SEARCH_HINT
+            SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT, SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT, SqlppHint.HASH_JOIN_HINT,
+            SqlppHint.USE_SECONDARY_INDEX_SEARCH_HINT, SqlppHint.SINGLE_DATASET_PREDICATE_SELECTIVITY_HINT
           );
           if (hintToken != null) {
             annotation = parseExpressionAnnotation(hintToken);
@@ -3632,6 +3725,7 @@
   boolean not = false;
   OperatorExpr op = null;
   Expression operand = null;
+  IExpressionAnnotation annotation = null;
 }
 {
     operand = ConcatExpr()
@@ -3639,6 +3733,10 @@
         LOOKAHEAD(2)
         (<NOT> { not = true; })? <LIKE>
         {
+          Token hintToken = fetchHint(token, SqlppHint.SINGLE_DATASET_PREDICATE_SELECTIVITY_HINT);
+          if (hintToken != null) {
+               annotation = parseExpressionAnnotation(hintToken);
+          }
           op = new OperatorExpr();
           op.addOperand(operand);
           op.setCurrentop(true);
@@ -3653,6 +3751,9 @@
           } catch (CompilationException e){
             throw new SqlppParseException(getSourceLocation(token), e.getMessage());
           }
+          if (annotation != null) {
+             op.addHint(annotation);
+          }
         }
 
         operand = ConcatExpr()
@@ -4675,12 +4776,33 @@
   Token startToken = null;
   SelectRegular selectRegular = null;
   SelectElement selectElement = null;
+  List<List<String>> fieldExclusions = new ArrayList<List<String>>();
+  List<String> nestedField = new ArrayList<String>();
+  String identifier;
   boolean distinct = false;
 }
 {
   <SELECT> { startToken = token; } (<ALL>|<DISTINCT> { distinct = true; } )?
   (
-    selectRegular = SelectRegular()
+    (
+      selectRegular = SelectRegular()
+      ( LOOKAHEAD({laIdentifier(EXCLUDE)}) <IDENTIFIER>
+        identifier = Identifier() { nestedField.add(identifier); }
+        ( <DOT> identifier = Identifier() { nestedField.add(identifier); } )*
+        {
+          fieldExclusions.add(nestedField);
+          nestedField = new ArrayList<String>();
+        }
+        ( LOOKAHEAD(1) // Force <COMMA> to be recognized for a nested field in our EXCLUDE list.
+          <COMMA> identifier = Identifier() { nestedField.add(identifier); }
+          ( <DOT> identifier = Identifier() { nestedField.add(identifier); } )*
+          {
+            fieldExclusions.add(nestedField);
+            nestedField = new ArrayList<String>();
+          }
+        )*
+      )?
+    )
     |
     selectElement = SelectElement()
   )?
@@ -4694,7 +4816,7 @@
         selectRegular = new SelectRegular(projections);
         selectRegular.setSourceLocation(sourceLoc);
     }
-    SelectClause selectClause = new SelectClause(selectElement, selectRegular, distinct);
+    SelectClause selectClause = new SelectClause(selectElement, selectRegular, fieldExclusions, distinct);
     selectClause.setSourceLocation(sourceLoc);
     return selectClause;
   }
@@ -4752,7 +4874,12 @@
   (
     <MUL> { kind = Projection.Kind.STAR; startSrcLoc = getSourceLocation(token); }
     | LOOKAHEAD(3) expr = VariableRef() <DOT> <MUL> { kind = Projection.Kind.VAR_STAR; }
-    | expr = Expression() ((<AS>)? name = Identifier())?
+    | expr = Expression()
+      ( // EXCLUDE is a soft-keyword-- we want to avoid mistaking EXCLUDE as an identifier here.
+        LOOKAHEAD({ getToken(1).kind == AS || getToken(1).kind == QUOTED_STRING
+                    || (getToken(1).kind == IDENTIFIER && !laIdentifier(1, EXCLUDE)) })
+        (<AS>)? name = Identifier()
+      )?
       {
         kind = Projection.Kind.NAMED_EXPR;
         if (name == null) {
@@ -5304,11 +5431,12 @@
 
     void CommonTokenAction(Token token) {
       Token hintToken = token.specialToken;
-      if (hintToken != null) {
+      while (hintToken != null) { // make this a while loop
         hintToken.sourceLocation = new SourceLocation(hintToken.beginLine, hintToken.beginColumn);
         String text = hintToken.image.substring(1).trim();
         boolean hintFound = hintToken.parseHint(text);
         hintCollector.put(hintToken.sourceLocation, hintFound ? hintToken.hint.getIdentifier() : hintToken.hintParams);
+        hintToken = hintToken.specialToken;
       }
     }
 }
@@ -5318,6 +5446,7 @@
 {
   <ADAPTER: "adapter">
   | <ALL : "all">
+  | <ANALYZE: "analyze">
   | <AND : "and">
   | <ANY : "any">
   | <APPLY : "apply">
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 fd0d30b..8880461 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
@@ -423,16 +423,16 @@
         if (createMetadataDataset) {
             final double bloomFilterFalsePositiveRate =
                     appContext.getStorageProperties().getBloomFilterFalsePositiveRate();
-            LSMBTreeLocalResourceFactory lsmBtreeFactory =
-                    new LSMBTreeLocalResourceFactory(storageComponentProvider.getStorageManager(), typeTraits,
-                            cmpFactories, null, null, null, opTrackerFactory, ioOpCallbackFactory,
-                            pageWriteCallbackFactory, storageComponentProvider.getMetadataPageManagerFactory(),
-                            new AsterixVirtualBufferCacheProvider(datasetId),
-                            storageComponentProvider.getIoOperationSchedulerProvider(),
-                            appContext.getMetadataMergePolicyFactory(),
-                            StorageConstants.DEFAULT_COMPACTION_POLICY_PROPERTIES, true, bloomFilterKeyFields,
-                            bloomFilterFalsePositiveRate, true, null, NoOpCompressorDecompressorFactory.INSTANCE, true,
-                            TypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
+            LSMBTreeLocalResourceFactory lsmBtreeFactory = new LSMBTreeLocalResourceFactory(
+                    storageComponentProvider.getStorageManager(), typeTraits, cmpFactories, null, null, null,
+                    opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory,
+                    storageComponentProvider.getMetadataPageManagerFactory(),
+                    new AsterixVirtualBufferCacheProvider(datasetId),
+                    storageComponentProvider.getIoOperationSchedulerProvider(),
+                    appContext.getMetadataMergePolicyFactory(), StorageConstants.DEFAULT_COMPACTION_POLICY_PROPERTIES,
+                    true, bloomFilterKeyFields, bloomFilterFalsePositiveRate, true, null,
+                    NoOpCompressorDecompressorFactory.INSTANCE, true,
+                    TypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE, false);
             DatasetLocalResourceFactory dsLocalResourceFactory =
                     new DatasetLocalResourceFactory(datasetId, lsmBtreeFactory);
             // TODO(amoudi) Creating the index should be done through the same code path as
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
index c0bdc75..9a2821e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
@@ -100,7 +100,7 @@
                         pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                         mergePolicyFactory, mergePolicyProperties, true, null, bloomFilterFalsePositiveRate,
                         index.isPrimaryIndex(), btreeFields, compDecompFactory, false,
-                        typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
+                        typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE, false);
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE,
                         dataset.getDatasetType().toString());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
index 168b56e..d119067 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
@@ -21,6 +21,7 @@
 import java.util.List;
 import java.util.Map;
 
+import org.apache.asterix.common.config.DatasetConfig;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import org.apache.asterix.common.context.IStorageComponentProvider;
@@ -92,13 +93,13 @@
                                 filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
                                 ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
                                 ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
-                                bloomFilterFalsePositiveRate, false, btreeFields, hasBloomFilter,
+                                bloomFilterFalsePositiveRate, btreeFields, hasBloomFilter,
                                 typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE)
                         : new ExternalBTreeWithBuddyLocalResourceFactory(storageManager, typeTraits, cmpFactories,
                                 filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
                                 ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
                                 ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
-                                bloomFilterFalsePositiveRate, false, btreeFields, hasBloomFilter,
+                                bloomFilterFalsePositiveRate, btreeFields, hasBloomFilter,
                                 typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
             case INTERNAL:
                 AsterixVirtualBufferCacheProvider vbcProvider =
@@ -112,12 +113,15 @@
                     compDecompFactory = NoOpCompressorDecompressorFactory.INSTANCE;
                 }
 
+                boolean isSecondaryNoIncrementalMaintenance = index.getIndexType() == DatasetConfig.IndexType.SAMPLE;
+
                 return new LSMBTreeLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
                         filterCmpFactories, filterFields, opTrackerFactory, ioOpCallbackFactory,
                         pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                         mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
                         bloomFilterFalsePositiveRate, index.isPrimaryIndex(), btreeFields, compDecompFactory,
-                        hasBloomFilter, typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
+                        hasBloomFilter, typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE,
+                        isSecondaryNoIncrementalMaintenance);
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE,
                         dataset.getDatasetType().toString());
@@ -127,7 +131,7 @@
     private static ITypeTraits[] getTypeTraits(MetadataProvider metadataProvider, Dataset dataset, Index index,
             ARecordType recordType, ARecordType metaType) throws AlgebricksException {
         ITypeTraits[] primaryTypeTraits = dataset.getPrimaryTypeTraits(metadataProvider, recordType, metaType);
-        if (index.isPrimaryIndex()) {
+        if (index.isPrimaryIndex() || index.getIndexType() == DatasetConfig.IndexType.SAMPLE) {
             return primaryTypeTraits;
         } else if (dataset.getDatasetType() == DatasetType.EXTERNAL
                 && index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
@@ -162,7 +166,7 @@
             Index index, ARecordType recordType, ARecordType metaType) throws AlgebricksException {
         IBinaryComparatorFactory[] primaryCmpFactories =
                 dataset.getPrimaryComparatorFactories(metadataProvider, recordType, metaType);
-        if (index.isPrimaryIndex()) {
+        if (index.isPrimaryIndex() || index.getIndexType() == DatasetConfig.IndexType.SAMPLE) {
             return dataset.getPrimaryComparatorFactories(metadataProvider, recordType, metaType);
         } else if (dataset.getDatasetType() == DatasetType.EXTERNAL
                 && index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
@@ -200,7 +204,8 @@
         if (index.isPrimaryIndex() || index.isPrimaryKeyIndex()) {
             return dataset.getPrimaryBloomFilterFields();
         }
-        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL
+                && index.getIndexType() != DatasetConfig.IndexType.SAMPLE) {
             if (index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
                 return FilesIndexDescription.BLOOM_FILTER_FIELDS;
             } else {
@@ -224,6 +229,8 @@
                     bloomFilterKeyFields[i] = i;
                 }
                 return bloomFilterKeyFields;
+            case SAMPLE:
+                return null;
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
                         String.valueOf(index.getIndexType()));
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 454d3da..95b0906 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
@@ -60,6 +60,7 @@
         public static final byte FEED = 0x02;
         public static final byte LOADABLE = 0x03;
         public static final byte FUNCTION = 0x04;
+        public static final byte SAMPLE = 0x05;
 
         // Hide implicit public constructor
         private Type() {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java
index 21b744f..324c660 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java
@@ -61,6 +61,7 @@
                 pp = getFeedPartitioningProperty(ds, domain, scanVariables);
                 break;
             case DataSource.Type.INTERNAL_DATASET:
+            case DataSource.Type.SAMPLE:
                 Set<LogicalVariable> pvars = new ListSet<>();
                 pp = getInternalDatasetPartitioningProperty(ds, domain, scanVariables, pvars);
                 propsLocal.add(new LocalOrderProperty(getOrderColumns(pvars)));
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 1d82b19..5daaaa0 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
@@ -53,6 +53,7 @@
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
 
 public class DatasetDataSource extends DataSource {
 
@@ -78,13 +79,18 @@
 
     private void initInternalDataset(IAType itemType, IAType metaItemType, IDatasetDetails datasetDetails)
             throws AlgebricksException {
-        InternalDatasetDetails internalDatasetDetails = (InternalDatasetDetails) datasetDetails;
+        schemaTypes =
+                createSchemaTypesForInternalDataset(itemType, metaItemType, (InternalDatasetDetails) datasetDetails);
+    }
+
+    static IAType[] createSchemaTypesForInternalDataset(IAType itemType, IAType metaItemType,
+            InternalDatasetDetails internalDatasetDetails) throws AlgebricksException {
         ARecordType recordType = (ARecordType) itemType;
         ARecordType metaRecordType = (ARecordType) metaItemType;
         List<IAType> partitioningKeyTypes =
                 KeyFieldTypeUtil.getPartitioningKeyTypes(internalDatasetDetails, recordType, metaRecordType);
         int n = partitioningKeyTypes.size();
-        schemaTypes = metaItemType == null ? new IAType[n + 1] : new IAType[n + 2];
+        IAType[] schemaTypes = metaItemType == null ? new IAType[n + 1] : new IAType[n + 2];
         for (int keyIndex = 0; keyIndex < n; ++keyIndex) {
             schemaTypes[keyIndex] = partitioningKeyTypes.get(keyIndex);
         }
@@ -92,11 +98,17 @@
         if (metaItemType != null) {
             schemaTypes[n + 1] = metaItemType;
         }
+        return schemaTypes;
     }
 
     private void initExternalDataset(IAType itemType) {
-        schemaTypes = new IAType[1];
+        schemaTypes = createSchemaTypesForExternalDataset(itemType);
+    }
+
+    static IAType[] createSchemaTypesForExternalDataset(IAType itemType) {
+        IAType[] schemaTypes = new IAType[1];
         schemaTypes[0] = itemType;
+        return schemaTypes;
     }
 
     @Override
@@ -135,7 +147,7 @@
                 return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false, null,
                         ((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null, true,
                         true, false, null, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory,
-                        outputLimit, false, false);
+                        outputLimit, false, false, DefaultTupleProjectorFactory.INSTANCE);
             default:
                 throw new AlgebricksException("Unknown datasource type");
         }
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 4cc83ab..87959c4 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
@@ -93,7 +93,9 @@
 import org.apache.asterix.metadata.lock.ExternalDatasetsRegistry;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.FullTextUtil;
+import org.apache.asterix.metadata.utils.IndexUtil;
 import org.apache.asterix.metadata.utils.MetadataConstants;
+import org.apache.asterix.metadata.utils.MetadataUtil;
 import org.apache.asterix.metadata.utils.SplitsAndConstraintsUtil;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionExtensionManager;
@@ -115,6 +117,7 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.core.algebra.base.Counter;
 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.expressions.IExpressionRuntimeProvider;
@@ -169,6 +172,7 @@
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
 import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
 
 public class MetadataProvider implements IMetadataProvider<DataSourceId, String> {
 
@@ -182,12 +186,11 @@
     private Dataverse defaultDataverse;
     private MetadataTransactionContext mdTxnCtx;
     private boolean isWriteTransaction;
-    private IAWriterFactory writerFactory;
     private FileSplit outputFile;
     private boolean asyncResults;
     private long maxResultReads;
     private ResultSetId resultSetId;
-    private IResultSerializerFactoryProvider resultSerializerFactoryProvider;
+    private Counter resultSetIdCounter;
     private TxnId txnId;
     private Map<String, Integer> externalDataLocks;
     private boolean blockingOperatorDisabled = false;
@@ -262,10 +265,6 @@
         this.isWriteTransaction = writeTransaction;
     }
 
-    public void setWriterFactory(IAWriterFactory writerFactory) {
-        this.writerFactory = writerFactory;
-    }
-
     public void setMetadataTxnContext(MetadataTransactionContext mdTxnCtx) {
         this.mdTxnCtx = mdTxnCtx;
     }
@@ -274,10 +273,6 @@
         return mdTxnCtx;
     }
 
-    public IAWriterFactory getWriterFactory() {
-        return this.writerFactory;
-    }
-
     public FileSplit getOutputFile() {
         return outputFile;
     }
@@ -310,12 +305,12 @@
         this.resultSetId = resultSetId;
     }
 
-    public void setResultSerializerFactoryProvider(IResultSerializerFactoryProvider rafp) {
-        this.resultSerializerFactoryProvider = rafp;
+    public Counter getResultSetIdCounter() {
+        return resultSetIdCounter;
     }
 
-    public IResultSerializerFactoryProvider getResultSerializerFactoryProvider() {
-        return resultSerializerFactoryProvider;
+    public void setResultSetIdCounter(Counter resultSetIdCounter) {
+        this.resultSetIdCounter = resultSetIdCounter;
     }
 
     public boolean isWriteTransaction() {
@@ -437,6 +432,16 @@
         return MetadataManagerUtil.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
     }
 
+    public Index findSampleIndex(DataverseName dataverseName, String datasetName) throws AlgebricksException {
+        Pair<String, String> sampleIndexNames = IndexUtil.getSampleIndexNames(datasetName);
+        Index sampleIndex = getIndex(dataverseName, datasetName, sampleIndexNames.first);
+        if (sampleIndex != null && sampleIndex.getPendingOp() == MetadataUtil.PENDING_NO_OP) {
+            return sampleIndex;
+        }
+        sampleIndex = getIndex(dataverseName, datasetName, sampleIndexNames.second);
+        return sampleIndex != null && sampleIndex.getPendingOp() == MetadataUtil.PENDING_NO_OP ? sampleIndex : null;
+    }
+
     public Triple<DataverseName, String, Boolean> resolveDatasetNameUsingSynonyms(DataverseName dataverseName,
             String datasetName, boolean includingViews) throws AlgebricksException {
         DataverseName dvName = getActiveDataverseName(dataverseName);
@@ -542,7 +547,8 @@
             int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
             boolean propagateFilter, IMissingWriterFactory nonFilterWriterFactory, int[] minFilterFieldIndexes,
             int[] maxFilterFieldIndexes, ITupleFilterFactory tupleFilterFactory, long outputLimit,
-            boolean isIndexOnlyPlan, boolean isPrimaryIndexPointSearch) throws AlgebricksException {
+            boolean isIndexOnlyPlan, boolean isPrimaryIndexPointSearch, ITupleProjectorFactory tupleProjectorFactory)
+            throws AlgebricksException {
         boolean isSecondary = true;
         Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
                 dataset.getDatasetName(), dataset.getDatasetName());
@@ -561,6 +567,12 @@
             case BTREE:
                 numSecondaryKeys = ((Index.ValueIndexDetails) theIndex.getIndexDetails()).getKeyFieldNames().size();
                 break;
+            case SAMPLE:
+                if (isIndexOnlyPlan) {
+                    throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, "");
+                }
+                numSecondaryKeys = 0;
+                break;
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
                         theIndex.getIndexType().toString());
@@ -601,12 +613,13 @@
                     ? new LSMBTreeBatchPointSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
                             highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput,
                             retainMissing, nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
-                            maxFilterFieldIndexes, tupleFilterFactory, outputLimit)
+                            maxFilterFieldIndexes, tupleFilterFactory, outputLimit, tupleProjectorFactory)
                     : new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields, highKeyFields,
                             lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
                             nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes,
                             propagateFilter, nonFilterWriterFactory, tupleFilterFactory, outputLimit,
-                            proceedIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
+                            proceedIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan,
+                            tupleProjectorFactory);
         } else {
             btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
                     highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
@@ -683,7 +696,8 @@
 
     @Override
     public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteFileRuntime(IDataSink sink,
-            int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc) {
+            int[] printColumns, IPrinterFactory[] printerFactories, IAWriterFactory writerFactory,
+            RecordDescriptor inputDesc) {
         FileSplitDataSink fsds = (FileSplitDataSink) sink;
         FileSplitSinkId fssi = fsds.getId();
         FileSplit fs = fssi.getFileSplit();
@@ -691,14 +705,15 @@
         String nodeId = fs.getNodeName();
 
         SinkWriterRuntimeFactory runtime =
-                new SinkWriterRuntimeFactory(printColumns, printerFactories, outFile, getWriterFactory(), inputDesc);
+                new SinkWriterRuntimeFactory(printColumns, printerFactories, outFile, writerFactory, inputDesc);
         AlgebricksPartitionConstraint apc = new AlgebricksAbsolutePartitionConstraint(new String[] { nodeId });
         return new Pair<>(runtime, apc);
     }
 
     @Override
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink,
-            int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc,
+            int[] printColumns, IPrinterFactory[] printerFactories, IAWriterFactory writerFactory,
+            IResultSerializerFactoryProvider resultSerializerFactoryProvider, RecordDescriptor inputDesc,
             IResultMetadata metadata, JobSpecification spec) throws AlgebricksException {
         ResultSetDataSink rsds = (ResultSetDataSink) sink;
         ResultSetSinkId rssId = rsds.getId();
@@ -706,7 +721,7 @@
         ResultWriterOperatorDescriptor resultWriter = null;
         try {
             IResultSerializerFactory resultSerializedAppenderFactory = resultSerializerFactoryProvider
-                    .getAqlResultSerializerFactoryProvider(printColumns, printerFactories, getWriterFactory());
+                    .getResultSerializerFactoryProvider(printColumns, printerFactories, writerFactory);
             resultWriter = new ResultWriterOperatorDescriptor(spec, rsId, metadata, getResultAsyncMode(),
                     resultSerializedAppenderFactory, getMaxResultReads());
         } catch (IOException e) {
@@ -859,7 +874,7 @@
      *
      * @param dataset
      * @return Number of elements that will be used to create a bloom filter per
-     *         dataset per partition
+     * dataset per partition
      * @throws AlgebricksException
      */
     public long getCardinalityPerPartitionHint(Dataset dataset) throws AlgebricksException {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/SampleDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/SampleDataSource.java
new file mode 100644
index 0000000..8ed3e16
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/SampleDataSource.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.metadata.declared;
+
+import java.util.List;
+
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.InternalDatasetDetails;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+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;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
+
+public class SampleDataSource extends DataSource {
+
+    private final Dataset dataset;
+
+    private final String sampleIndexName;
+
+    public SampleDataSource(Dataset dataset, String sampleIndexName, IAType itemType, IAType metaItemType,
+            INodeDomain domain) throws AlgebricksException {
+        super(createSampleDataSourceId(dataset, sampleIndexName), itemType, metaItemType, Type.SAMPLE, domain);
+        this.dataset = dataset;
+        this.sampleIndexName = sampleIndexName;
+        this.schemaTypes = DatasetDataSource.createSchemaTypesForInternalDataset(itemType, metaItemType,
+                (InternalDatasetDetails) dataset.getDatasetDetails());
+    }
+
+    @Override
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
+            MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
+            List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
+            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
+            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
+            IProjectionInfo<?> projectionInfo) throws AlgebricksException {
+        return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false, null, dataset,
+                sampleIndexName, null, null, true, true, false, null, null, null, tupleFilterFactory, outputLimit,
+                false, false, DefaultTupleProjectorFactory.INSTANCE);
+    }
+
+    @Override
+    public boolean isScanAccessPathALeaf() {
+        return false;
+    }
+
+    private static DataSourceId createSampleDataSourceId(Dataset dataset, String sampleIndexName) {
+        return new DataSourceId(dataset.getDataverseName(), dataset.getDatasetName(), new String[] { sampleIndexName });
+    }
+}
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 37e22ed..e3935a4 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
@@ -497,6 +497,10 @@
                         recordType, metaType, mergePolicyFactory, mergePolicyProperties, filterTypeTraits,
                         filterCmpFactories);
                 break;
+            case SAMPLE:
+                resourceFactory = BTreeResourceFactoryProvider.INSTANCE.getResourceFactory(mdProvider, this, index,
+                        recordType, metaType, mergePolicyFactory, mergePolicyProperties, null, null);
+                break;
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
                         index.getIndexType().toString());
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 9e67292..21d2aaa 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
@@ -275,6 +275,7 @@
         switch (indexType) {
             case ARRAY:
             case BTREE:
+            case SAMPLE:
                 return ResourceType.LSM_BTREE;
             case RTREE:
                 return ResourceType.LSM_RTREE;
@@ -296,7 +297,8 @@
     public enum IndexCategory {
         VALUE,
         TEXT,
-        ARRAY;
+        ARRAY,
+        SAMPLE;
 
         public static IndexCategory of(IndexType indexType) {
             switch (indexType) {
@@ -310,6 +312,8 @@
                     return TEXT;
                 case ARRAY:
                     return ARRAY;
+                case SAMPLE:
+                    return SAMPLE;
                 default:
                     throw new IllegalArgumentException(String.valueOf(indexType));
             }
@@ -533,6 +537,75 @@
         }
     }
 
+    public static class SampleIndexDetails extends AbstractIndexDetails {
+
+        private static final long serialVersionUID = 1L;
+
+        private final List<List<String>> keyFieldNames;
+
+        private final List<Integer> keyFieldSourceIndicators;
+
+        private final List<IAType> keyFieldTypes;
+
+        private final int sampleCardinalityTarget;
+
+        private final long sourceCardinality;
+
+        private final int sourceAvgItemSize;
+
+        private final long sampleSeed;
+
+        public SampleIndexDetails(List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators,
+                List<IAType> keyFieldTypes, int sampleCardinalityTarget, long sourceCardinality, int sourceAvgItemSize,
+                long sampleSeed) {
+            this.keyFieldNames = keyFieldNames;
+            this.keyFieldSourceIndicators = keyFieldSourceIndicators;
+            this.keyFieldTypes = keyFieldTypes;
+            this.sampleCardinalityTarget = sampleCardinalityTarget;
+            this.sourceCardinality = sourceCardinality;
+            this.sourceAvgItemSize = sourceAvgItemSize;
+            this.sampleSeed = sampleSeed;
+        }
+
+        @Override
+        IndexCategory getIndexCategory() {
+            return IndexCategory.SAMPLE;
+        }
+
+        public List<List<String>> getKeyFieldNames() {
+            return keyFieldNames;
+        }
+
+        public List<Integer> getKeyFieldSourceIndicators() {
+            return keyFieldSourceIndicators;
+        }
+
+        public List<IAType> getKeyFieldTypes() {
+            return keyFieldTypes;
+        }
+
+        @Override
+        public boolean isOverridingKeyFieldTypes() {
+            return false;
+        }
+
+        public int getSampleCardinalityTarget() {
+            return sampleCardinalityTarget;
+        }
+
+        public long getSourceCardinality() {
+            return sourceCardinality;
+        }
+
+        public int getSourceAvgItemSize() {
+            return sourceAvgItemSize;
+        }
+
+        public long getSampleSeed() {
+            return sampleSeed;
+        }
+    }
+
     @Deprecated
     private static Index.IIndexDetails createSimpleIndexDetails(IndexType indexType, List<List<String>> keyFieldNames,
             List<Integer> keyFieldSourceIndicators, List<IAType> keyFieldTypes, boolean overrideKeyFieldTypes,
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index bb51748..9c742ed 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -47,6 +47,7 @@
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.ACollectionCursor;
 import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AInt64;
 import org.apache.asterix.om.base.AInt8;
 import org.apache.asterix.om.base.AMutableInt8;
 import org.apache.asterix.om.base.ANull;
@@ -90,6 +91,10 @@
     public static final String INDEX_SEARCHKEY_ELEMENTS_FIELD_NAME = "SearchKeyElements";
     public static final String COMPLEXSEARCHKEY_UNNEST_FIELD_NAME = "UnnestList";
     public static final String COMPLEXSEARCHKEY_PROJECT_FIELD_NAME = "ProjectList";
+    public static final String SAMPLE_SEED = "SampleSeed";
+    public static final String SAMPLE_CARDINALITY_TARGET = "SampleCardinalityTarget";
+    public static final String SOURCE_CARDINALITY = "SourceCardinality";
+    public static final String SOURCE_AVG_ITEM_SIZE = "SourceAvgItemSize";
 
     protected final TxnId txnId;
     protected final MetadataNode metadataNode;
@@ -240,6 +245,9 @@
                     searchElements.add(searchElement);
                 }
                 break;
+            case SAMPLE:
+                searchElements = Collections.emptyList();
+                break;
             default:
                 throw new AsterixException(ErrorCode.METADATA_ERROR, indexType.toString());
         }
@@ -452,6 +460,39 @@
                 }
                 indexDetails = new Index.ArrayIndexDetails(elementList, isOverridingKeyTypes);
                 break;
+            case SAMPLE:
+                keyFieldNames =
+                        searchElements.stream().map(Pair::getSecond).map(l -> l.get(0)).collect(Collectors.toList());
+                keyFieldTypes = searchKeyType.stream().map(l -> l.get(0)).collect(Collectors.toList());
+
+                int sampleSeedPos = indexRecord.getType().getFieldIndex(SAMPLE_SEED);
+                if (sampleSeedPos < 0) {
+                    throw new AsterixException(ErrorCode.METADATA_ERROR, SAMPLE_SEED);
+                }
+                long sampleSeed = ((AInt64) indexRecord.getValueByPos(sampleSeedPos)).getLongValue();
+
+                int sampleCardinalityTargetPos = indexRecord.getType().getFieldIndex(SAMPLE_CARDINALITY_TARGET);
+                if (sampleCardinalityTargetPos < 0) {
+                    throw new AsterixException(ErrorCode.METADATA_ERROR, SAMPLE_CARDINALITY_TARGET);
+                }
+                int sampleCardinalityTarget =
+                        ((AInt32) indexRecord.getValueByPos(sampleCardinalityTargetPos)).getIntegerValue();
+
+                int sourceCardinalityPos = indexRecord.getType().getFieldIndex(SOURCE_CARDINALITY);
+                if (sourceCardinalityPos < 0) {
+                    throw new AsterixException(ErrorCode.METADATA_ERROR, SOURCE_CARDINALITY);
+                }
+                long sourceCardinality = ((AInt64) indexRecord.getValueByPos(sourceCardinalityPos)).getLongValue();
+
+                int sourceAvgItemSizePos = indexRecord.getType().getFieldIndex(SOURCE_AVG_ITEM_SIZE);
+                if (sourceAvgItemSizePos < 0) {
+                    throw new AsterixException(ErrorCode.METADATA_ERROR, SOURCE_AVG_ITEM_SIZE);
+                }
+                int sourceAvgItemSize = ((AInt32) indexRecord.getValueByPos(sourceAvgItemSizePos)).getIntegerValue();
+
+                indexDetails = new Index.SampleIndexDetails(keyFieldNames, keyFieldSourceIndicator, keyFieldTypes,
+                        sampleCardinalityTarget, sourceCardinality, sourceAvgItemSize, sampleSeed);
+                break;
             default:
                 throw new AsterixException(ErrorCode.METADATA_ERROR, indexType.toString());
         }
@@ -527,6 +568,9 @@
                 // If we have a complex index, we persist all of the names in the complex SK name array instead.
                 searchKey = Collections.emptyList();
                 break;
+            case SAMPLE:
+                searchKey = ((Index.SampleIndexDetails) index.getIndexDetails()).getKeyFieldNames();
+                break;
             default:
                 throw new AsterixException(ErrorCode.METADATA_ERROR, indexType.toString());
         }
@@ -596,6 +640,7 @@
         writeSearchKeySourceIndicator(index);
         writeExcludeUnknownKey(index);
         writeCast(index);
+        writeSampleDetails(index);
     }
 
     private void writeComplexSearchKeys(Index.ArrayIndexDetails indexDetails) throws HyracksDataException {
@@ -767,6 +812,9 @@
                 keySourceIndicator = ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
                         .map(Index.ArrayIndexElement::getSourceIndicator).collect(Collectors.toList());
                 break;
+            case SAMPLE:
+                keySourceIndicator = ((Index.SampleIndexDetails) index.getIndexDetails()).getKeyFieldSourceIndicators();
+                break;
             default:
                 throw new AsterixException(ErrorCode.METADATA_ERROR, index.getIndexType().toString());
         }
@@ -855,4 +903,38 @@
             }
         }
     }
+
+    private void writeSampleDetails(Index index) throws HyracksDataException {
+        if (index.getIndexType() == IndexType.SAMPLE) {
+            Index.SampleIndexDetails indexDetails = (Index.SampleIndexDetails) index.getIndexDetails();
+
+            nameValue.reset();
+            fieldValue.reset();
+            aString.setValue(SAMPLE_SEED);
+            stringSerde.serialize(aString, nameValue.getDataOutput());
+            int64Serde.serialize(new AInt64(indexDetails.getSampleSeed()), fieldValue.getDataOutput());
+            recordBuilder.addField(nameValue, fieldValue);
+
+            nameValue.reset();
+            fieldValue.reset();
+            aString.setValue(SAMPLE_CARDINALITY_TARGET);
+            stringSerde.serialize(aString, nameValue.getDataOutput());
+            int32Serde.serialize(new AInt32(indexDetails.getSampleCardinalityTarget()), fieldValue.getDataOutput());
+            recordBuilder.addField(nameValue, fieldValue);
+
+            nameValue.reset();
+            fieldValue.reset();
+            aString.setValue(SOURCE_CARDINALITY);
+            stringSerde.serialize(aString, nameValue.getDataOutput());
+            int64Serde.serialize(new AInt64(indexDetails.getSourceCardinality()), fieldValue.getDataOutput());
+            recordBuilder.addField(nameValue, fieldValue);
+
+            nameValue.reset();
+            fieldValue.reset();
+            aString.setValue(SOURCE_AVG_ITEM_SIZE);
+            stringSerde.serialize(aString, nameValue.getDataOutput());
+            int32Serde.serialize(new AInt32(indexDetails.getSourceAvgItemSize()), fieldValue.getDataOutput());
+            recordBuilder.addField(nameValue, fieldValue);
+        }
+    }
 }
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 92390a7..0ee9516 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
@@ -30,7 +30,6 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.MetadataException;
-import org.apache.asterix.common.exceptions.NoOpWarningCollector;
 import org.apache.asterix.common.external.IDataSourceAdapter;
 import org.apache.asterix.common.external.IDataSourceAdapter.AdapterType;
 import org.apache.asterix.common.functions.ExternalFunctionLanguage;
@@ -62,6 +61,7 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
 
 /**
  * A utility class for providing helper functions for feeds TODO: Refactor this
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
index 5fd96a6..ce98008 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
@@ -174,7 +174,7 @@
         if (fnKind != FunctionKind.SCALAR) {
             return false;
         }
-        if (!(fnInfo instanceof IExternalFunctionInfo)) {
+        if (!fnInfo.isExternal()) {
             return false;
         }
         ExternalFunctionLanguage language = ((IExternalFunctionInfo) fnInfo).getLanguage();
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 b899e16..c6eafe0 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
@@ -30,7 +30,6 @@
 import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
 import org.apache.asterix.common.config.DatasetConfig.TransactionState;
 import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.exceptions.NoOpWarningCollector;
 import org.apache.asterix.external.api.ITypedAdapterFactory;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.external.indexing.IndexingConstants;
@@ -64,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.NoOpWarningCollector;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ISecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ISecondaryIndexOperationsHelper.java
new file mode 100644
index 0000000..13ff19d
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ISecondaryIndexOperationsHelper.java
@@ -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.
+ */
+
+package org.apache.asterix.metadata.utils;
+
+import java.util.Set;
+
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
+
+public interface ISecondaryIndexOperationsHelper {
+
+    void init() throws AlgebricksException;
+
+    JobSpecification buildCreationJobSpec() throws AlgebricksException;
+
+    JobSpecification buildLoadingJobSpec() throws AlgebricksException;
+
+    JobSpecification buildCompactJobSpec() throws AlgebricksException;
+
+    JobSpecification buildDropJobSpec(Set<IndexDropOperatorDescriptor.DropOption> options) throws AlgebricksException;
+
+    IFileSplitProvider getSecondaryFileSplitProvider();
+
+    RecordDescriptor getSecondaryRecDesc();
+
+    IBinaryComparatorFactory[] getSecondaryComparatorFactories();
+
+    AlgebricksPartitionConstraint getSecondaryPartitionConstraint();
+}
\ No newline at end of file
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 ad92803..7e9cfc3 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
@@ -79,6 +79,8 @@
         } else if (index.getIndexType() == DatasetConfig.IndexType.ARRAY) {
             numSecondaryKeys = ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
                     .map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
+        } else if (index.getIndexType() == DatasetConfig.IndexType.SAMPLE) {
+            return null;
         } else {
             throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, index.getIndexType().toString());
         }
@@ -111,6 +113,8 @@
             case SINGLE_PARTITION_NGRAM_INVIX:
             case SINGLE_PARTITION_WORD_INVIX:
                 break;
+            case SAMPLE:
+                break;
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
                         index.getIndexType().toString());
@@ -120,21 +124,21 @@
 
     public static JobSpecification buildDropIndexJobSpec(Index index, MetadataProvider metadataProvider,
             Dataset dataset, SourceLocation sourceLoc) throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper =
+        ISecondaryIndexOperationsHelper secondaryIndexHelper =
                 SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
         return secondaryIndexHelper.buildDropJobSpec(EnumSet.noneOf(DropOption.class));
     }
 
     public static JobSpecification buildDropIndexJobSpec(Index index, MetadataProvider metadataProvider,
             Dataset dataset, Set<DropOption> options, SourceLocation sourceLoc) throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper =
+        ISecondaryIndexOperationsHelper secondaryIndexHelper =
                 SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
         return secondaryIndexHelper.buildDropJobSpec(options);
     }
 
     public static JobSpecification buildSecondaryIndexCreationJobSpec(Dataset dataset, Index index,
             MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper =
+        ISecondaryIndexOperationsHelper secondaryIndexHelper =
                 SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
         return secondaryIndexHelper.buildCreationJobSpec();
     }
@@ -147,8 +151,8 @@
     public static JobSpecification buildSecondaryIndexLoadingJobSpec(Dataset dataset, Index index,
             MetadataProvider metadataProvider, List<ExternalFile> files, SourceLocation sourceLoc)
             throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper;
-        if (dataset.isCorrelated()) {
+        ISecondaryIndexOperationsHelper secondaryIndexHelper;
+        if (dataset.isCorrelated() && supportsCorrelated(index.getIndexType())) { //TODO:REVISIT
             secondaryIndexHelper = SecondaryCorrelatedTreeIndexOperationsHelper.createIndexOperationsHelper(dataset,
                     index, metadataProvider, sourceLoc);
         } else {
@@ -156,14 +160,18 @@
                     metadataProvider, sourceLoc);
         }
         if (files != null) {
-            secondaryIndexHelper.setExternalFiles(files);
+            ((SecondaryIndexOperationsHelper) secondaryIndexHelper).setExternalFiles(files);
         }
         return secondaryIndexHelper.buildLoadingJobSpec();
     }
 
+    private static boolean supportsCorrelated(DatasetConfig.IndexType indexType) {
+        return indexType != DatasetConfig.IndexType.SAMPLE;
+    }
+
     public static JobSpecification buildSecondaryIndexCompactJobSpec(Dataset dataset, Index index,
             MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper =
+        ISecondaryIndexOperationsHelper secondaryIndexHelper =
                 SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
         return secondaryIndexHelper.buildCompactJobSpec();
     }
@@ -240,4 +248,9 @@
     public static boolean excludesUnknowns(Index index) {
         return !includesUnknowns(index);
     }
+
+    public static Pair<String, String> getSampleIndexNames(String datasetName) {
+        return new Pair<>(MetadataConstants.SAMPLE_INDEX_1_PREFIX + datasetName,
+                MetadataConstants.SAMPLE_INDEX_2_PREFIX + datasetName);
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
index b52cddd..0f0ea40 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
@@ -57,6 +57,11 @@
     public static final String FULL_TEXT_CONFIG_DATASET_NAME = "FullTextConfig";
     public static final String FULL_TEXT_FILTER_DATASET_NAME = "FullTextFilter";
 
+    public static final String PRIMARY_INDEX_PREFIX = "primary_idx_";
+    public static final String SAMPLE_INDEX_PREFIX = "sample_idx_";
+    public static final String SAMPLE_INDEX_1_PREFIX = SAMPLE_INDEX_PREFIX + "1_";
+    public static final String SAMPLE_INDEX_2_PREFIX = SAMPLE_INDEX_PREFIX + "2_";
+
     private MetadataConstants() {
     }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
index 7bb11e5..1de387f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
@@ -311,6 +311,20 @@
     }
 
     @Override
+    public void analyzeDatasetBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
+            String datasetName) throws AlgebricksException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireDatasetCreateIndexLock(locks, dataverseName, datasetName);
+    }
+
+    @Override
+    public void analyzeDatasetDropBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
+            String datasetName) throws AlgebricksException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireDatasetWriteLock(locks, dataverseName, datasetName);
+    }
+
+    @Override
     public void compactBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
             String datasetName) throws AlgebricksException {
         lockMgr.acquireDataverseReadLock(locks, dataverseName);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SampleOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SampleOperationsHelper.java
new file mode 100644
index 0000000..0d3e015
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SampleOperationsHelper.java
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.metadata.utils;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.config.OptimizationConfUtil;
+import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.asterix.common.utils.StorageConstants;
+import org.apache.asterix.formats.base.IDataFormat;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.aggregates.collections.FirstElementEvalFactory;
+import org.apache.asterix.runtime.evaluators.comparisons.GreaterThanDescriptor;
+import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorDescriptor;
+import org.apache.asterix.runtime.operators.StreamStatsOperatorDescriptor;
+import org.apache.asterix.runtime.runningaggregates.std.SampleSlotRunningAggregateFunctionFactory;
+import org.apache.asterix.runtime.runningaggregates.std.TidRunningAggregateDescriptor;
+import org.apache.asterix.runtime.utils.RuntimeUtils;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
+import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
+import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
+import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
+import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.evaluators.ColumnAccessEvalFactory;
+import org.apache.hyracks.algebricks.runtime.operators.aggreg.SimpleAlgebricksAccumulatingAggregatorFactory;
+import org.apache.hyracks.algebricks.runtime.operators.aggrun.RunningAggregateRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.SinkRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
+import org.apache.hyracks.algebricks.runtime.operators.std.StreamProjectRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.std.StreamSelectRuntimeFactory;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+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.connectors.OneToOneConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.dataflow.std.group.AbstractAggregatorDescriptorFactory;
+import org.apache.hyracks.dataflow.std.group.sort.SortGroupByOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.api.IIndexBuilderFactory;
+import org.apache.hyracks.storage.am.common.build.IndexBuilderFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IndexCreateOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.common.IResourceFactory;
+
+/**
+ * Utility class for sampling operations.
+ * <p>
+ * The sampling method described in:
+ * "A Convenient Algorithm for Drawing a Simple Random Sample",
+ * by A. I. McLeod and D. R. Bellhouse
+ */
+public class SampleOperationsHelper implements ISecondaryIndexOperationsHelper {
+
+    public static final String DATASET_STATS_OPERATOR_NAME = "Sample.DatasetStats";
+
+    private final MetadataProvider metadataProvider;
+    private final Dataset dataset;
+    private final Index index;
+    private final SourceLocation sourceLoc;
+
+    private ARecordType itemType;
+    private ARecordType metaType;
+    private RecordDescriptor recordDesc;
+    private IBinaryComparatorFactory[] comparatorFactories;
+    private IFileSplitProvider fileSplitProvider;
+    private AlgebricksPartitionConstraint partitionConstraint;
+    private ILSMMergePolicyFactory mergePolicyFactory;
+    private Map<String, String> mergePolicyProperties;
+    private int groupbyNumFrames;
+
+    protected SampleOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) {
+        this.dataset = dataset;
+        this.index = index;
+        this.metadataProvider = metadataProvider;
+        this.sourceLoc = sourceLoc;
+    }
+
+    @Override
+    public void init() throws AlgebricksException {
+        itemType =
+                (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+        metaType = DatasetUtil.getMetaType(metadataProvider, dataset);
+        recordDesc = dataset.getPrimaryRecordDescriptor(metadataProvider);
+        comparatorFactories = dataset.getPrimaryComparatorFactories(metadataProvider, itemType, metaType);
+        groupbyNumFrames = getGroupByNumFrames(metadataProvider, sourceLoc);
+
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint =
+                metadataProvider.getSplitProviderAndConstraints(dataset, index.getIndexName());
+        fileSplitProvider = secondarySplitsAndConstraint.first;
+        partitionConstraint = secondarySplitsAndConstraint.second;
+
+        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+        mergePolicyFactory = compactionInfo.first;
+        mergePolicyProperties = compactionInfo.second;
+    }
+
+    @Override
+    public JobSpecification buildCreationJobSpec() throws AlgebricksException {
+        JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
+        IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
+        IResourceFactory resourceFactory = dataset.getResourceFactory(metadataProvider, index, itemType, metaType,
+                mergePolicyFactory, mergePolicyProperties);
+        IIndexBuilderFactory indexBuilderFactory = new IndexBuilderFactory(storageComponentProvider.getStorageManager(),
+                fileSplitProvider, resourceFactory, true);
+        IndexCreateOperatorDescriptor indexCreateOp = new IndexCreateOperatorDescriptor(spec, indexBuilderFactory);
+        indexCreateOp.setSourceLocation(sourceLoc);
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp, partitionConstraint);
+        spec.addRoot(indexCreateOp);
+        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+        return spec;
+    }
+
+    @Override
+    public JobSpecification buildLoadingJobSpec() throws AlgebricksException {
+        Index.SampleIndexDetails indexDetails = (Index.SampleIndexDetails) index.getIndexDetails();
+        int sampleCardinalityTarget = indexDetails.getSampleCardinalityTarget();
+        long sampleSeed = indexDetails.getSampleSeed();
+        IDataFormat format = metadataProvider.getDataFormat();
+        int nFields = recordDesc.getFieldCount();
+        int[] columns = new int[nFields];
+        for (int i = 0; i < nFields; i++) {
+            columns[i] = i;
+        }
+
+        JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
+        IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
+                metadataProvider.getStorageComponentProvider().getStorageManager(), fileSplitProvider);
+
+        // job spec:
+        IndexUtil.bindJobEventListener(spec, metadataProvider);
+
+        // dummy key provider ----> primary index scan
+        IOperatorDescriptor sourceOp = DatasetUtil.createDummyKeyProviderOp(spec, dataset, metadataProvider);
+        IOperatorDescriptor targetOp = DatasetUtil.createPrimaryIndexScanOp(spec, metadataProvider, dataset);
+        spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+        sourceOp = targetOp;
+
+        // primary index scan ----> stream stats op
+        targetOp = new StreamStatsOperatorDescriptor(spec, recordDesc, DATASET_STATS_OPERATOR_NAME);
+        spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+        sourceOp = targetOp;
+
+        // stream stats op ----> (running agg + select)
+        // ragg produces a slot number and a tuple counter for each tuple
+        // If the slot number is 0 then the tuple is not in the sample and is removed by subsequent select op.
+        // If the slot number is greater than 0 then the tuple is in the sample.
+        // There could be several tuples with the same slot number, the latest one wins
+        // (with the greatest tuple counter). This is accomplished by the group by below
+        BuiltinType raggSlotType = BuiltinType.AINT32;
+        BuiltinType raggCounterType = BuiltinType.AINT64;
+        int[] raggProjectColumns = new int[nFields + 2];
+        raggProjectColumns[0] = nFields;
+        raggProjectColumns[1] = nFields + 1;
+        System.arraycopy(columns, 0, raggProjectColumns, 2, nFields);
+        int[] raggAggColumns = { nFields, nFields + 1 };
+
+        ISerializerDeserializerProvider serdeProvider = format.getSerdeProvider();
+        ISerializerDeserializer[] raggSerdes = new ISerializerDeserializer[nFields + 2];
+        raggSerdes[0] = serdeProvider.getSerializerDeserializer(raggSlotType);
+        raggSerdes[1] = serdeProvider.getSerializerDeserializer(raggCounterType);
+        System.arraycopy(recordDesc.getFields(), 0, raggSerdes, 2, nFields);
+
+        ITypeTraitProvider typeTraitProvider = format.getTypeTraitProvider();
+        ITypeTraits[] raggTraits = new ITypeTraits[nFields + 2];
+        raggTraits[0] = typeTraitProvider.getTypeTrait(raggSlotType);
+        raggTraits[1] = typeTraitProvider.getTypeTrait(raggCounterType);
+        System.arraycopy(recordDesc.getTypeTraits(), 0, raggTraits, 2, nFields);
+
+        RecordDescriptor raggRecordDesc = new RecordDescriptor(raggSerdes, raggTraits);
+
+        IRunningAggregateEvaluatorFactory raggSlotEvalFactory =
+                new SampleSlotRunningAggregateFunctionFactory(sampleCardinalityTarget, sampleSeed);
+        IRunningAggregateEvaluatorFactory raggCounterEvalFactory = TidRunningAggregateDescriptor.FACTORY
+                .createFunctionDescriptor().createRunningAggregateEvaluatorFactory(new IScalarEvaluatorFactory[0]);
+        RunningAggregateRuntimeFactory raggRuntimeFactory =
+                new RunningAggregateRuntimeFactory(raggProjectColumns, raggAggColumns,
+                        new IRunningAggregateEvaluatorFactory[] { raggSlotEvalFactory, raggCounterEvalFactory });
+
+        IFunctionDescriptor gtDescriptor = GreaterThanDescriptor.FACTORY.createFunctionDescriptor();
+        gtDescriptor.setImmutableStates(raggSlotType, raggSlotType);
+        IScalarEvaluatorFactory gtFactory =
+                gtDescriptor.createEvaluatorFactory(new IScalarEvaluatorFactory[] { new ColumnAccessEvalFactory(0),
+                        format.getConstantEvalFactory(new AsterixConstantValue(new AInt32(0))) });
+        StreamSelectRuntimeFactory selectRuntimeFactory = new StreamSelectRuntimeFactory(gtFactory, null,
+                format.getBinaryBooleanInspectorFactory(), false, -1, null);
+
+        targetOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
+                new IPushRuntimeFactory[] { raggRuntimeFactory, selectRuntimeFactory },
+                new RecordDescriptor[] { raggRecordDesc, raggRecordDesc });
+        spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+        sourceOp = targetOp;
+
+        // (running agg + select) ---> group-by
+        int[] groupFields = new int[] { 0 }; // [slot]
+        int[] sortFields = new int[] { 0, 1 }; // [slot, counter]
+        OrderOperator.IOrder sortSlotOrder = OrderOperator.ASC_ORDER;
+        OrderOperator.IOrder sortCounterOrder = OrderOperator.DESC_ORDER;
+        IBinaryComparatorFactoryProvider comparatorFactoryProvider = format.getBinaryComparatorFactoryProvider();
+        IBinaryComparatorFactory[] raggCmpFactories = {
+                comparatorFactoryProvider.getBinaryComparatorFactory(raggSlotType,
+                        sortSlotOrder.getKind() == OrderOperator.IOrder.OrderKind.ASC),
+                comparatorFactoryProvider.getBinaryComparatorFactory(raggCounterType,
+                        sortCounterOrder.getKind() == OrderOperator.IOrder.OrderKind.ASC) };
+
+        INormalizedKeyComputerFactoryProvider normKeyProvider = format.getNormalizedKeyComputerFactoryProvider();
+        INormalizedKeyComputerFactory[] normKeyFactories = {
+                normKeyProvider.getNormalizedKeyComputerFactory(raggSlotType,
+                        sortSlotOrder.getKind() == OrderOperator.IOrder.OrderKind.ASC),
+                normKeyProvider.getNormalizedKeyComputerFactory(raggCounterType,
+                        sortCounterOrder.getKind() == OrderOperator.IOrder.OrderKind.ASC) };
+
+        // agg = [counter, .. original columns ..]
+        IAggregateEvaluatorFactory[] aggFactories = new IAggregateEvaluatorFactory[nFields + 1];
+        for (int i = 0; i < aggFactories.length; i++) {
+            aggFactories[i] = new FirstElementEvalFactory(
+                    new IScalarEvaluatorFactory[] { new ColumnAccessEvalFactory(1 + i) }, false, sourceLoc);
+        }
+        AbstractAggregatorDescriptorFactory aggregatorFactory =
+                new SimpleAlgebricksAccumulatingAggregatorFactory(aggFactories, groupFields);
+
+        targetOp = new SortGroupByOperatorDescriptor(spec, groupbyNumFrames, sortFields, groupFields, normKeyFactories,
+                raggCmpFactories, aggregatorFactory, aggregatorFactory, raggRecordDesc, raggRecordDesc, false);
+        spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+        sourceOp = targetOp;
+
+        // group by --> project (remove ragg fields)
+        int[] projectColumns = new int[nFields];
+        for (int i = 0; i < nFields; i++) {
+            projectColumns[i] = 2 + i;
+        }
+        StreamProjectRuntimeFactory projectRuntimeFactory = new StreamProjectRuntimeFactory(projectColumns);
+        targetOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1, new IPushRuntimeFactory[] { projectRuntimeFactory },
+                new RecordDescriptor[] { recordDesc });
+        spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+        sourceOp = targetOp;
+
+        // project ---> bulk load op
+        targetOp = createTreeIndexBulkLoadOp(spec, columns, dataflowHelperFactory,
+                StorageConstants.DEFAULT_TREE_FILL_FACTOR, sampleCardinalityTarget);
+        spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+        sourceOp = targetOp;
+
+        // bulk load op ----> sink op
+        SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+        sinkRuntimeFactory.setSourceLocation(sourceLoc);
+        targetOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0, new IPushRuntimeFactory[] { sinkRuntimeFactory },
+                new RecordDescriptor[] { recordDesc });
+        spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+
+        spec.addRoot(targetOp);
+        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+
+        return spec;
+    }
+
+    protected LSMIndexBulkLoadOperatorDescriptor createTreeIndexBulkLoadOp(JobSpecification spec,
+            int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor,
+            long numElementHint) {
+        LSMIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new LSMIndexBulkLoadOperatorDescriptor(spec,
+                recordDesc, fieldPermutation, fillFactor, false, numElementHint, true, dataflowHelperFactory, null,
+                LSMIndexBulkLoadOperatorDescriptor.BulkLoadUsage.LOAD, dataset.getDatasetId(), null);
+        treeIndexBulkLoadOp.setSourceLocation(sourceLoc);
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
+                partitionConstraint);
+        return treeIndexBulkLoadOp;
+    }
+
+    @Override
+    public JobSpecification buildDropJobSpec(Set<IndexDropOperatorDescriptor.DropOption> options)
+            throws AlgebricksException {
+        return SecondaryTreeIndexOperationsHelper.buildDropJobSpecImpl(dataset, index, options, metadataProvider,
+                sourceLoc);
+    }
+
+    @Override
+    public JobSpecification buildCompactJobSpec() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public IFileSplitProvider getSecondaryFileSplitProvider() {
+        return fileSplitProvider;
+    }
+
+    @Override
+    public RecordDescriptor getSecondaryRecDesc() {
+        return recordDesc;
+    }
+
+    @Override
+    public IBinaryComparatorFactory[] getSecondaryComparatorFactories() {
+        return comparatorFactories;
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getSecondaryPartitionConstraint() {
+        return partitionConstraint;
+    }
+
+    private static int getGroupByNumFrames(MetadataProvider metadataProvider, SourceLocation sourceLoc)
+            throws AlgebricksException {
+        return OptimizationConfUtil.getGroupByNumFrames(
+                metadataProvider.getApplicationContext().getCompilerProperties(), metadataProvider.getConfig(),
+                sourceLoc);
+    }
+}
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 9e0d650..694b153 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
@@ -91,7 +91,7 @@
 // TODO: We should eventually have a hierarchy of classes that can create all
 // possible index job specs,
 // not just for creation.
-public abstract class SecondaryIndexOperationsHelper {
+public abstract class SecondaryIndexOperationsHelper implements ISecondaryIndexOperationsHelper {
     protected final MetadataProvider metadataProvider;
     protected final Dataset dataset;
     protected final Index index;
@@ -160,10 +160,10 @@
                 metadataProvider.getConfig(), sourceLoc);
     }
 
-    public static SecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
+    public static ISecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
             MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
 
-        SecondaryIndexOperationsHelper indexOperationsHelper;
+        ISecondaryIndexOperationsHelper indexOperationsHelper;
         switch (index.getIndexType()) {
             case ARRAY:
                 indexOperationsHelper =
@@ -182,6 +182,9 @@
                 indexOperationsHelper =
                         new SecondaryInvertedIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
                 break;
+            case SAMPLE:
+                indexOperationsHelper = new SampleOperationsHelper(dataset, index, metadataProvider, sourceLoc);
+                break;
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, sourceLoc,
                         index.getIndexType());
@@ -202,7 +205,8 @@
 
     protected abstract int getNumSecondaryKeys();
 
-    protected void init() throws AlgebricksException {
+    @Override
+    public void init() throws AlgebricksException {
         payloadSerde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
         metaSerde =
                 metaType == null ? null : SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(metaType);
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 0373e32..e8825fc 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
@@ -72,13 +72,19 @@
 
     @Override
     public JobSpecification buildDropJobSpec(Set<DropOption> options) throws AlgebricksException {
+        return buildDropJobSpecImpl(dataset, index, options, metadataProvider, sourceLoc);
+    }
+
+    static JobSpecification buildDropJobSpecImpl(Dataset dataset, Index index, Set<DropOption> dropOptions,
+            MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
         JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
                 metadataProvider.getSplitProviderAndConstraints(dataset, index.getIndexName());
         IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
                 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);
+        IndexDropOperatorDescriptor btreeDrop =
+                new IndexDropOperatorDescriptor(spec, dataflowHelperFactory, dropOptions);
         btreeDrop.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeDrop,
                 splitsAndConstraint.second);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ABoolean.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ABoolean.java
index 5589205..0664fec 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ABoolean.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ABoolean.java
@@ -50,7 +50,7 @@
 
     @Override
     public String toString() {
-        return Boolean.toString(bVal).toUpperCase();
+        return Boolean.toString(bVal);
     }
 
     @Override
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
index d964079..cb6c5fe 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
@@ -20,6 +20,7 @@
 
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.hyracks.util.JSONUtil;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ArrayNode;
@@ -89,7 +90,7 @@
                 if (i > 0) {
                     sb.append(", ");
                 }
-                sb.append(type.getFieldNames()[i]);
+                JSONUtil.quoteAndEscape(sb, type.getFieldNames()[i]);
                 sb.append(": ");
                 sb.append(fields[i]);
             }
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 0c9f0dd..56971eb 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
@@ -1799,28 +1799,26 @@
         addPrivateFunction(MAKE_FIELD_NAME_HANDLE, AnyTypeComputer.INSTANCE, true);
 
         // cast null type constructors
-        addPrivateFunction(BOOLEAN_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_BOOLEAN, true);
-        addPrivateFunction(INT8_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT8, true);
-        addPrivateFunction(INT16_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT16, true);
-        addPrivateFunction(INT32_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT32, true);
-        addPrivateFunction(INT64_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT64, true);
-        addPrivateFunction(FLOAT_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_FLOAT, true);
-        addPrivateFunction(DOUBLE_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DOUBLE, true);
-        addPrivateFunction(STRING_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_STRING, true);
-        addPrivateFunction(DATE_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DATE, true);
-        addPrivateFunction(DATE_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT, NullableTypeComputer.INSTANCE_DATE, true);
-        addPrivateFunction(TIME_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_TIME, true);
-        addPrivateFunction(TIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT, NullableTypeComputer.INSTANCE_TIME, true);
-        addPrivateFunction(DATETIME_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DATE_TIME, true);
-        addPrivateFunction(DATETIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT, NullableTypeComputer.INSTANCE_DATE_TIME,
+        addFunction(BOOLEAN_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_BOOLEAN, true);
+        addFunction(INT8_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT8, true);
+        addFunction(INT16_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT16, true);
+        addFunction(INT32_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT32, true);
+        addFunction(INT64_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT64, true);
+        addFunction(FLOAT_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_FLOAT, true);
+        addFunction(DOUBLE_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DOUBLE, true);
+        addFunction(STRING_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_STRING, true);
+        addFunction(DATE_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DATE, true);
+        addFunction(DATE_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT, NullableTypeComputer.INSTANCE_DATE, true);
+        addFunction(TIME_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_TIME, true);
+        addFunction(TIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT, NullableTypeComputer.INSTANCE_TIME, true);
+        addFunction(DATETIME_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DATE_TIME, true);
+        addFunction(DATETIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT, NullableTypeComputer.INSTANCE_DATE_TIME, true);
+        addFunction(DURATION_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DURATION, true);
+        addFunction(DAY_TIME_DURATION_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DAY_TIME_DURATION, true);
+        addFunction(YEAR_MONTH_DURATION_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_YEAR_MONTH_DURATION,
                 true);
-        addPrivateFunction(DURATION_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DURATION, true);
-        addPrivateFunction(DAY_TIME_DURATION_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DAY_TIME_DURATION,
-                true);
-        addPrivateFunction(YEAR_MONTH_DURATION_DEFAULT_NULL_CONSTRUCTOR,
-                NullableTypeComputer.INSTANCE_YEAR_MONTH_DURATION, true);
-        addPrivateFunction(UUID_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_UUID, true);
-        addPrivateFunction(BINARY_BASE64_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_BINARY, true);
+        addFunction(UUID_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_UUID, true);
+        addFunction(BINARY_BASE64_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_BINARY, true);
 
         addPrivateFunction(NUMERIC_UNARY_MINUS, NumericUnaryTypeComputer.INSTANCE, true);
         addPrivateFunction(NUMERIC_SUBTRACT, NumericAddSubMulDivTypeComputer.INSTANCE_SUB, true);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java
index a3ec9c6..8a2cefa 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java
@@ -101,4 +101,9 @@
     public boolean getNullCall() {
         return nullCall;
     }
+
+    @Override
+    public boolean isExternal() {
+        return true;
+    }
 }
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 83af00d..bdb7277 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
@@ -42,6 +42,7 @@
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeHelper;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -84,6 +85,7 @@
             case STRING:
                 String fn = ((AString) fieldName).getStringValue();
                 fieldNameSet.add(fn);
+                pathList.add(List.of(fn));
                 break;
             case ARRAY:
                 AOrderedList pathOrdereList = (AOrderedList) fieldName;
@@ -237,16 +239,18 @@
         IAType[] fieldTypes = inputRecordType.getFieldTypes();
 
         for (int i = 0; i < fieldNames.length; i++) {
+            IAType originalType = fieldTypes[i];
+            IAType actualType = TypeComputeUtils.getActualType(originalType);
             if (!fieldNameSet.contains(fieldNames[i])) { // The main field is to be kept
                 addField(inputRecordType, fieldNames[i], resultFieldNames, resultFieldTypes);
-            } else if (!pathList.isEmpty() && fieldTypes[i].getTypeTag() == ATypeTag.OBJECT) {
-                ARecordType subRecord = (ARecordType) fieldTypes[i];
+            } else if (!pathList.isEmpty() && actualType.getTypeTag() == ATypeTag.OBJECT) {
+                ARecordType subRecord = (ARecordType) actualType;
                 fieldPathStack.push(fieldNames[i]);
-                subRecord = deepCheckAndCopy(fieldPathStack, subRecord, pathList, inputRecordType.isOpen());
+                subRecord = deepCheckAndCopy(fieldPathStack, subRecord, pathList, subRecord.isOpen());
                 fieldPathStack.pop();
                 if (subRecord != null) {
                     resultFieldNames.add(fieldNames[i]);
-                    resultFieldTypes.add(subRecord);
+                    resultFieldTypes.add(wrapWithOriginalType(subRecord, originalType));
                 }
             }
         }
@@ -259,6 +263,18 @@
 
     }
 
+    private static IAType wrapWithOriginalType(IAType typeToModify, IAType originalType) {
+        if (TypeHelper.canBeMissing(originalType) && !TypeHelper.canBeNull(originalType)) {
+            return AUnionType.createMissableType(typeToModify);
+        } else if (!TypeHelper.canBeMissing(originalType) && TypeHelper.canBeNull(originalType)) {
+            return AUnionType.createNullableType(typeToModify);
+        } else if (TypeHelper.canBeUnknown(originalType)) {
+            return AUnionType.createUnknownableType(typeToModify);
+        } else {
+            return typeToModify;
+        }
+    }
+
     /**
      * Comparison elements of two paths
      * Note: l2 uses a LIFO insert and removal.
@@ -314,12 +330,14 @@
         for (int i = 0; i < srcFieldNames.length; i++) {
             fieldPath.push(srcFieldNames[i]);
             if (!isRemovePath(fieldPath, pathList)) {
-                if (srcFieldTypes[i].getTypeTag() == ATypeTag.OBJECT) {
-                    ARecordType subRecord = (ARecordType) srcFieldTypes[i];
+                IAType originalType = srcFieldTypes[i];
+                IAType actualType = TypeComputeUtils.getActualType(originalType);
+                if (actualType.getTypeTag() == ATypeTag.OBJECT) {
+                    ARecordType subRecord = (ARecordType) actualType;
                     subRecord = deepCheckAndCopy(fieldPath, subRecord, pathList, isOpen);
                     if (subRecord != null) {
                         destFieldNames.add(srcFieldNames[i]);
-                        destFieldTypes.add(subRecord);
+                        destFieldTypes.add(wrapWithOriginalType(subRecord, originalType));
                     }
                 } else {
                     destFieldNames.add(srcFieldNames[i]);
diff --git a/asterixdb/asterix-docker/docker/.gitattributes b/asterixdb/asterix-podman/docker/.gitattributes
similarity index 100%
rename from asterixdb/asterix-docker/docker/.gitattributes
rename to asterixdb/asterix-podman/docker/.gitattributes
diff --git a/asterixdb/asterix-docker/docker/Dockerfile b/asterixdb/asterix-podman/docker/Dockerfile
similarity index 100%
rename from asterixdb/asterix-docker/docker/Dockerfile
rename to asterixdb/asterix-podman/docker/Dockerfile
diff --git a/asterixdb/asterix-docker/docker/asterix-configuration.xml b/asterixdb/asterix-podman/docker/asterix-configuration.xml
similarity index 100%
rename from asterixdb/asterix-docker/docker/asterix-configuration.xml
rename to asterixdb/asterix-podman/docker/asterix-configuration.xml
diff --git a/asterixdb/asterix-docker/docker/fbm.adm b/asterixdb/asterix-podman/docker/fbm.adm
similarity index 100%
rename from asterixdb/asterix-docker/docker/fbm.adm
rename to asterixdb/asterix-podman/docker/fbm.adm
diff --git a/asterixdb/asterix-docker/docker/fbu.adm b/asterixdb/asterix-podman/docker/fbu.adm
similarity index 100%
rename from asterixdb/asterix-docker/docker/fbu.adm
rename to asterixdb/asterix-podman/docker/fbu.adm
diff --git a/asterixdb/asterix-docker/docker/supervisord.conf b/asterixdb/asterix-podman/docker/supervisord.conf
similarity index 100%
rename from asterixdb/asterix-docker/docker/supervisord.conf
rename to asterixdb/asterix-podman/docker/supervisord.conf
diff --git a/asterixdb/asterix-docker/docker/twm.adm b/asterixdb/asterix-podman/docker/twm.adm
similarity index 100%
rename from asterixdb/asterix-docker/docker/twm.adm
rename to asterixdb/asterix-podman/docker/twm.adm
diff --git a/asterixdb/asterix-docker/docker/twu.adm b/asterixdb/asterix-podman/docker/twu.adm
similarity index 100%
rename from asterixdb/asterix-docker/docker/twu.adm
rename to asterixdb/asterix-podman/docker/twu.adm
diff --git a/asterixdb/asterix-podman/pom.xml b/asterixdb/asterix-podman/pom.xml
new file mode 100644
index 0000000..3d32518
--- /dev/null
+++ b/asterixdb/asterix-podman/pom.xml
@@ -0,0 +1,156 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT 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/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>apache-asterixdb</artifactId>
+    <groupId>org.apache.asterix</groupId>
+    <version>0.9.8-SNAPSHOT</version>
+  </parent>
+  <artifactId>asterix-podman</artifactId>
+    <dependencies>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
+            <artifactId>asterix-server</artifactId>
+            <version>${project.version}</version>
+            <type>deb</type>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.asterix</groupId>
+            <artifactId>asterix-app</artifactId>
+            <version>${project.version}</version>
+            <classifier>tests</classifier>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.asterix</groupId>
+            <artifactId>asterix-test-framework</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>testcontainers</artifactId>
+            <version>1.17.1</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+  <properties>
+    <root.dir>${basedir}/..</root.dir>
+  </properties>
+
+  <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>
+   <build>
+       <plugins>
+           <plugin>
+               <groupId>org.apache.rat</groupId>
+               <artifactId>apache-rat-plugin</artifactId>
+               <configuration>
+                   <excludes combine.children="append">
+                       <exclude>src/test/resources/setup.sh</exclude>
+                       <exclude>src/test/resources/passwd</exclude>
+                       <exclude>src/test/resources/socktest/Containerfile</exclude>
+                       <exclude>src/test/resources/testenv.conf</exclude>
+                   </excludes>
+               </configuration>
+           </plugin>
+       </plugins>
+   </build>
+    <profiles>
+        <profile>
+            <id>podman.tests</id>
+            <properties>
+                <test.excludes>**/*.java</test.excludes>
+                <itest.includes>**/PodmanPythonFunctionIT.java</itest.includes>
+                <failIfNoTests>false</failIfNoTests>
+            </properties>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>nl.lexemmens</groupId>
+                        <artifactId>podman-maven-plugin</artifactId>
+                        <version>1.8.0</version>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>build</goal>
+                                </goals>
+                                <phase>generate-test-resources</phase>
+                            </execution>
+                        </executions>
+                        <configuration>
+                            <skipAuth>true</skipAuth>
+                            <images>
+                                <image>
+                                    <name>asterixdb/socktest</name>
+                                    <build>
+                                        <pull>false</pull>
+                                        <createLatestTag>true</createLatestTag>
+                                        <containerFileDir>src/test/resources/socktest</containerFileDir>
+                                    </build>
+                                </image>
+                            </images>
+                        </configuration>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-resources-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>copy-external-data-resources</id>
+                                <phase>generate-resources</phase>
+                                <goals>
+                                    <goal>copy-resources</goal>
+                                </goals>
+                                <configuration>
+                                    <outputDirectory>target/</outputDirectory>
+                                    <overwrite>true</overwrite>
+                                    <resources>
+                                        <resource>
+                                            <directory>../asterix-server/target</directory>
+                                            <includes>
+                                                <include>asterix-server*.deb</include>
+                                            </includes>
+                                        </resource>
+                                    </resources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+
+</project>
diff --git a/asterixdb/asterix-podman/src/test/java/org/apache/asterix/test/podman/PodmanPythonFunctionIT.java b/asterixdb/asterix-podman/src/test/java/org/apache/asterix/test/podman/PodmanPythonFunctionIT.java
new file mode 100644
index 0000000..f0f89cd
--- /dev/null
+++ b/asterixdb/asterix-podman/src/test/java/org/apache/asterix/test/podman/PodmanPythonFunctionIT.java
@@ -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.
+ */
+package org.apache.asterix.test.podman;
+
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.test.runtime.ExecutionTestUtil;
+import org.apache.asterix.test.runtime.LangExecutionUtil;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+import org.testcontainers.DockerClientFactory;
+import org.testcontainers.containers.BindMode;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.utility.DockerImageName;
+
+import com.github.dockerjava.api.DockerClient;
+
+/**
+ * Runs the Python UDF tests within a container using domain sockets.
+ */
+@RunWith(Parameterized.class)
+public class PodmanPythonFunctionIT {
+    public static final DockerImageName ASTERIX_IMAGE = DockerImageName.parse("asterixdb/socktest");
+    @ClassRule
+    public static GenericContainer<?> asterix = new GenericContainer(ASTERIX_IMAGE).withExposedPorts(19004, 5006, 19002)
+            .withFileSystemBind("../asterix-app/", "/var/tmp/asterix-app/", BindMode.READ_WRITE);
+    protected static final String TEST_CONFIG_FILE_NAME = "../asterix-app/src/test/resources/cc.conf";
+    private static final boolean cleanupOnStop = true;
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        final TestExecutor testExecutor = new TestExecutor(
+                List.of(InetSocketAddress.createUnresolved(asterix.getHost(), asterix.getMappedPort(19002))));
+        asterix.execInContainer("/opt/setup.sh");
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor, false, true, new PodmanUDFLibrarian(asterix));
+        setEndpoints(testExecutor);
+        testExecutor.waitForClusterActive(60, TimeUnit.SECONDS);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        try {
+        } finally {
+            ExecutionTestUtil.tearDown(cleanupOnStop);
+            DockerClient dc = DockerClientFactory.instance().client();
+            dc.removeImageCmd(ASTERIX_IMAGE.asCanonicalNameString()).withForce(true).exec();
+        }
+    }
+
+    @Parameters(name = "PodmanPythonFunctionIT {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only_sqlpp.xml", "testsuite_it_python.xml",
+                "../asterix-app/src/test/resources/runtimets");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public PodmanPythonFunctionIT(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+
+    private static void setEndpoints(TestExecutor testExecutor) {
+        final Map<String, InetSocketAddress> ncEndPoints = new HashMap<>();
+        final String ip = asterix.getHost();
+        final String nodeId = "asterix_nc";
+        int apiPort = asterix.getMappedPort(19004);
+        ncEndPoints.put(nodeId, InetSocketAddress.createUnresolved(ip, apiPort));
+        testExecutor.setNcEndPoints(ncEndPoints);
+    }
+}
diff --git a/asterixdb/asterix-podman/src/test/java/org/apache/asterix/test/podman/PodmanUDFLibrarian.java b/asterixdb/asterix-podman/src/test/java/org/apache/asterix/test/podman/PodmanUDFLibrarian.java
new file mode 100644
index 0000000..025f607
--- /dev/null
+++ b/asterixdb/asterix-podman/src/test/java/org/apache/asterix/test/podman/PodmanUDFLibrarian.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.podman;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.asterix.app.external.IExternalUDFLibrarian;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.GenericContainer;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.json.JsonReadFeature;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+public class PodmanUDFLibrarian implements IExternalUDFLibrarian {
+    final GenericContainer<?> asterix;
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+    public PodmanUDFLibrarian(GenericContainer asterix) {
+        OBJECT_MAPPER.configure(JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS.mappedFeature(), true);
+        this.asterix = asterix;
+    }
+
+    @Override
+    public void install(URI path, String type, String libPath, Pair<String, String> credentials) throws Exception {
+        Container.ExecResult curlResult = null;
+        int retryCt = 0;
+        while (retryCt < 10) {
+            try {
+                curlResult = asterix.execInContainer("curl", "--no-progress-meter", "-X", "POST", "-u",
+                        credentials.first + ":" + credentials.second, "-F",
+                        "data=@" + "/var/tmp/asterix-app/" + libPath, "-F", "type=" + type,
+                        "http://localhost:19004" + path.getRawPath());
+                handleResponse(curlResult);
+                return;
+            } catch (RuntimeException e) {
+                retryCt++;
+                if (retryCt > 9)
+                    throw e;
+            }
+        }
+    }
+
+    @Override
+    public void uninstall(URI path, Pair<String, String> credentials) throws IOException, AsterixException {
+        try {
+            Container.ExecResult curlResult = asterix.execInContainer("curl", "-X", "DELETE", "-u",
+                    credentials.first + ":" + credentials.second, "http://localhost:19004" + path.getPath());
+            handleResponse(curlResult);
+        } catch (InterruptedException e) {
+            throw new IOException(e);
+        }
+    }
+
+    private void handleResponse(Container.ExecResult result) throws AsterixException, JsonProcessingException {
+        if (result.getExitCode() != 0) {
+            throw new AsterixException(result.getStderr());
+        }
+        JsonNode resp = OBJECT_MAPPER.readTree(result.getStdout().replace('\0', ' '));
+        if (resp.has("error")) {
+            throw new AsterixException(resp.get("error").toString());
+        }
+        return;
+    }
+}
diff --git a/asterixdb/asterix-podman/src/test/resources/cc.conf b/asterixdb/asterix-podman/src/test/resources/cc.conf
new file mode 100644
index 0000000..e4cbd73
--- /dev/null
+++ b/asterixdb/asterix-podman/src/test/resources/cc.conf
@@ -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.
+
+[nc/asterix_nc1]
+txn.log.dir=/opt/apache-asterixdb/data/txnlog
+core.dump.dir=/opt/apache-asterixdb/logs/coredump
+iodevices=/opt/apache-asterixdb/data/
+nc.api.port=19004
+
+[nc]
+address=127.0.0.1
+command=asterixnc
+credential.file=/opt/apache-asterixdb/etc/passwd
+jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:5006
+python.ds.path = /tmp/pyudf.socket
+
+[cc]
+address = 127.0.0.1
+
+[common]
+log.level = INFO
+log.dir = /opt/apache-asterixdb/logs/
diff --git a/asterixdb/asterix-podman/src/test/resources/passwd b/asterixdb/asterix-podman/src/test/resources/passwd
new file mode 100644
index 0000000..a1ea5b0
--- /dev/null
+++ b/asterixdb/asterix-podman/src/test/resources/passwd
@@ -0,0 +1 @@
+admin:$2a$12$JxgDzf/uOn1NS2Y3exhrDOf7JY/eUHQH7HeH90s5Ye2gALoO0FsQy
diff --git a/asterixdb/asterix-podman/src/test/resources/setup.sh b/asterixdb/asterix-podman/src/test/resources/setup.sh
new file mode 100644
index 0000000..e3523aa
--- /dev/null
+++ b/asterixdb/asterix-podman/src/test/resources/setup.sh
@@ -0,0 +1,8 @@
+#!/bin/bash
+cd /var/tmp/asterix-app/
+shiv -o target/TweetSent.pyz --site-packages src/test/resources/TweetSent scikit-learn
+cp -a /var/tmp/asterix-app/data/classifications /opt/apache-asterixdb/data/
+cp -a /var/tmp/asterix-app/data/twitter /opt/apache-asterixdb/data/
+cp -a /var/tmp/asterix-app/data/big-object /opt/apache-asterixdb/data/
+mkdir -p /opt/apache-asterixdb/target/data/
+cp -a /var/tmp/asterix-app/target/data/big-object /opt/apache-asterixdb/target/data/
\ No newline at end of file
diff --git a/asterixdb/asterix-podman/src/test/resources/socktest/Containerfile b/asterixdb/asterix-podman/src/test/resources/socktest/Containerfile
new file mode 100644
index 0000000..a7546d5
--- /dev/null
+++ b/asterixdb/asterix-podman/src/test/resources/socktest/Containerfile
@@ -0,0 +1,17 @@
+FROM ubuntu:22.04
+RUN apt -y update
+RUN DEBIAN_FRONTEND=noninteractive TZ=Etc/UTC apt -y install systemd openjdk-17-jre-headless unzip wget curl python3-pip python3-venv python3-systemd
+RUN pip3 install shiv msgpack
+COPY target/asterix-server_*all.deb .
+RUN dpkg -i asterix-server*.deb
+COPY src/test/resources/cc.conf /opt/apache-asterixdb/cc.conf
+COPY src/test/resources/passwd /opt/apache-asterixdb/etc/passwd
+RUN mkdir -p /etc/systemd/system/pyudf@.service.d/
+COPY src/test/resources/testenv.conf /etc/systemd/system/pyudf@.service.d/
+COPY src/test/resources/setup.sh /opt
+RUN chmod +x /opt/setup.sh
+RUN systemctl enable asterix-nc asterix-cc pyudf.socket
+
+EXPOSE 19001 19002 19004
+
+CMD [ "/lib/systemd/systemd" ]
diff --git a/asterixdb/asterix-podman/src/test/resources/testenv.conf b/asterixdb/asterix-podman/src/test/resources/testenv.conf
new file mode 100644
index 0000000..0c2f182
--- /dev/null
+++ b/asterixdb/asterix-podman/src/test/resources/testenv.conf
@@ -0,0 +1,3 @@
+[Service]
+Environment="FOO=BAR=BAZ"
+Environment="BAR=BAZ"
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 b787e26..6c5c0c7 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
@@ -32,14 +32,14 @@
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-class FirstElementEvalFactory implements IAggregateEvaluatorFactory {
+public class FirstElementEvalFactory implements IAggregateEvaluatorFactory {
 
     private static final long serialVersionUID = 1L;
     private final IScalarEvaluatorFactory[] args;
     private final boolean isLocal;
     private final SourceLocation sourceLoc;
 
-    FirstElementEvalFactory(IScalarEvaluatorFactory[] args, boolean isLocal, SourceLocation sourceLoc) {
+    public FirstElementEvalFactory(IScalarEvaluatorFactory[] args, boolean isLocal, SourceLocation sourceLoc) {
         this.args = args;
         this.isLocal = isLocal;
         this.sourceLoc = sourceLoc;
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 c8d4824..1a453a7 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
@@ -33,6 +33,7 @@
 import org.apache.asterix.om.pointables.PointableAllocator;
 import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
 import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
 import org.apache.asterix.om.types.AOrderedListType;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
@@ -170,8 +171,9 @@
                 int pos = runtimeRecordTypeInfo.getFieldIndex(fieldNamePointable.getByteArray(),
                         fieldNamePointable.getStartOffset() + 1, fieldNamePointable.getLength() - 1);
                 if (pos >= 0) { // Closed field
-                    if (PointableHelper.sameType(ATypeTag.OBJECT, fieldTypePointable)) {
-                        processRecord((ARecordType) requiredType.getFieldTypes()[pos],
+                    if (PointableHelper.sameType(ATypeTag.OBJECT, fieldTypePointable)
+                            && PointableHelper.sameType(ATypeTag.OBJECT, fieldValuePointable)) {
+                        processRecord((ARecordType) TypeComputeUtils.getActualType(requiredType.getFieldTypes()[pos]),
                                 (ARecordVisitablePointable) fieldValuePointable, inputList, nestedLevel + 1);
                         tabvs.reset();
                         rbStack.get(nestedLevel + 1).write(tabvs.getDataOutput(), true);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
index bc763bd..f96a6e9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
@@ -300,6 +300,8 @@
             IAType type1 = (IAType) context.getType(le);
             if (type0.getTypeTag().equals(ATypeTag.ANY)) {
                 type0 = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+            } else if (type0.getTypeTag().equals(ATypeTag.UNION)) {
+                type0 = ((AUnionType) type0).getActualType();
             }
             if (type1.getTypeTag().equals(ATypeTag.ANY)) {
                 type1 = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StreamStats.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StreamStats.java
new file mode 100644
index 0000000..3d1979f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StreamStats.java
@@ -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.
+ */
+
+package org.apache.asterix.runtime.operators;
+
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+
+/**
+ * Helper method to access stats produced by {@link org.apache.asterix.runtime.operators.StreamStatsOperatorDescriptor}
+ */
+public final class StreamStats {
+
+    private final long cardinality;
+
+    private final int avgTupleSize;
+
+    public StreamStats(IOperatorStats opStats) {
+        this.cardinality = opStats.getTupleCounter().get();
+        long totalTupleSize = opStats.getPageReads().get();
+        this.avgTupleSize = cardinality > 0 ? (int) (totalTupleSize / cardinality) : 0;
+    }
+
+    static void update(IOperatorStats opStats, long tupleCount, long tupleSize) {
+        opStats.getTupleCounter().update(tupleCount);
+        opStats.getPageReads().update(tupleSize);
+    }
+
+    public long getCardinality() {
+        return cardinality;
+    }
+
+    public int getAvgTupleSize() {
+        return avgTupleSize;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StreamStatsOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StreamStatsOperatorDescriptor.java
new file mode 100644
index 0000000..2e8832a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StreamStatsOperatorDescriptor.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.runtime.operators;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.IStatsCollector;
+import org.apache.hyracks.api.job.profiling.OperatorStats;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+
+/**
+ * Computes total tuple count and total tuple length for all input tuples,
+ * and emits these values as operator stats.
+ */
+public final class StreamStatsOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+    private static final long serialVersionUID = 2L;
+
+    private final String operatorName;
+
+    public StreamStatsOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor rDesc,
+            String operatorName) {
+        super(spec, 1, 1);
+        outRecDescs[0] = rDesc;
+        this.operatorName = operatorName;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+
+        return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+
+            private FrameTupleAccessor fta;
+            private long totalTupleCount;
+            private long totalTupleLength;
+
+            @Override
+            public void open() throws HyracksDataException {
+                fta = new FrameTupleAccessor(outRecDescs[0]);
+                totalTupleCount = 0;
+                writer.open();
+                IStatsCollector coll = ctx.getStatsCollector();
+                if (coll != null) {
+                    coll.add(new OperatorStats(operatorName));
+                }
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                fta.reset(buffer);
+                computeStats();
+                FrameUtils.flushFrame(buffer, writer);
+            }
+
+            private void computeStats() {
+                int n = fta.getTupleCount();
+                totalTupleCount += n;
+                for (int i = 0; i < n; i++) {
+                    totalTupleLength += fta.getTupleLength(i);
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+                writer.fail();
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                IStatsCollector statsCollector = ctx.getStatsCollector();
+                if (statsCollector != null) {
+                    IOperatorStats stats = statsCollector.getOperatorStats(operatorName);
+                    StreamStats.update(stats, totalTupleCount, totalTupleLength);
+                }
+                writer.close();
+            }
+
+            @Override
+            public void flush() throws HyracksDataException {
+                writer.flush();
+            }
+
+            @Override
+            public String getDisplayName() {
+                return operatorName;
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/SampleSlotRunningAggregateFunctionFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/SampleSlotRunningAggregateFunctionFactory.java
new file mode 100644
index 0000000..a4bda44
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/SampleSlotRunningAggregateFunctionFactory.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.runningaggregates.std;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+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.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * See {@code SampleOperationsHelper} for the sampling algorithm details.
+ */
+public class SampleSlotRunningAggregateFunctionFactory implements IRunningAggregateEvaluatorFactory {
+
+    private static final long serialVersionUID = 2L;
+
+    private final int sampleCardinalityTarget;
+
+    private final long sampleSeed;
+
+    public SampleSlotRunningAggregateFunctionFactory(int sampleCardinalityTarget, long sampleSeed) {
+        this.sampleCardinalityTarget = sampleCardinalityTarget;
+        this.sampleSeed = sampleSeed;
+    }
+
+    @Override
+    public IRunningAggregateEvaluator createRunningAggregateEvaluator(IEvaluatorContext ctx)
+            throws HyracksDataException {
+
+        int sampleCardinalityTargetPerPartition = getSampleCardinalityTargetPerPartition(sampleCardinalityTarget,
+                ctx.getTaskContext().getPartitionCount());
+
+        return new IRunningAggregateEvaluator() {
+
+            private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+            private final DataOutput resultOutput = resultStorage.getDataOutput();
+            @SuppressWarnings("unchecked")
+            private final ISerializerDeserializer<AInt32> int32Serde =
+                    SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+            private final AMutableInt32 aInt32 = new AMutableInt32(0);
+
+            private final Random rnd = new Random(sampleSeed);
+            private long counter;
+
+            @Override
+            public void init() {
+                counter = 0;
+            }
+
+            @Override
+            public void step(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+                try {
+                    counter++;
+                    int outValue = evaluate();
+
+                    resultStorage.reset();
+                    aInt32.setValue(outValue);
+                    int32Serde.serialize(aInt32, resultOutput);
+                    result.set(resultStorage);
+                } catch (IOException e) {
+                    throw HyracksDataException.create(e);
+                }
+            }
+
+            private int evaluate() {
+                if (counter <= sampleCardinalityTargetPerPartition) {
+                    return (int) counter;
+                } else {
+                    long v = 1 + (long) (rnd.nextDouble() * counter);
+                    return v <= sampleCardinalityTargetPerPartition ? (int) v : 0;
+                }
+            }
+        };
+    }
+
+    private static int getSampleCardinalityTargetPerPartition(int sampleCardinalityTarget, int nPartitions) {
+        return Math.max(1, sampleCardinalityTarget / nPartitions + Math.min(sampleCardinalityTarget % nPartitions, 1));
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
index 95de235..c39db40 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
@@ -48,8 +48,7 @@
 
             @SuppressWarnings("unchecked")
             @Override
-            public IRunningAggregateEvaluator createRunningAggregateEvaluator(IEvaluatorContext ctx)
-                    throws HyracksDataException {
+            public IRunningAggregateEvaluator createRunningAggregateEvaluator(IEvaluatorContext ctx) {
 
                 return new IRunningAggregateEvaluator() {
 
@@ -57,7 +56,7 @@
                     private final ISerializerDeserializer<AInt64> serde =
                             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
                     private final AMutableInt64 m = new AMutableInt64(0);
-                    private int cnt;
+                    private long cnt;
 
                     @Override
                     public void step(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
@@ -69,7 +68,7 @@
                     }
 
                     @Override
-                    public void init() throws HyracksDataException {
+                    public void init() {
                         cnt = 1;
                     }
                 };
diff --git a/asterixdb/asterix-server/pom.xml b/asterixdb/asterix-server/pom.xml
index 16edbb8..c82384d 100644
--- a/asterixdb/asterix-server/pom.xml
+++ b/asterixdb/asterix-server/pom.xml
@@ -353,12 +353,30 @@
             </override>
             <override>
               <gavs>
+                <gav>io.opencensus:opencensus-proto:0.2.0</gav>
+              </gavs>
+              <url>https://raw.githubusercontent.com/census-instrumentation/opencensus-proto/v0.2.0/LICENSE</url>
+            </override>
+            <override>
+              <gavs>
                 <gav>io.opencensus:opencensus-api:0.31.1</gav>
                 <gav>io.opencensus:opencensus-contrib-http-util:0.31.1</gav>
               </gavs>
               <url>https://raw.githubusercontent.com/census-instrumentation/opencensus-java/v0.31.1/LICENSE</url>
             </override>
             <override>
+              <gavs>
+                <gav>io.opencensus:opencensus-contrib-exemplar-util:0.31.0</gav>
+                <gav>io.opencensus:opencensus-contrib-grpc-metrics:0.31.0</gav>
+                <gav>io.opencensus:opencensus-contrib-resource-util:0.31.0</gav>
+                <gav>io.opencensus:opencensus-exporter-metrics-util:0.31.0</gav>
+                <gav>io.opencensus:opencensus-exporter-stats-stackdriver:0.31.0</gav>
+                <gav>io.opencensus:opencensus-impl-core:0.31.0</gav>
+                <gav>io.opencensus:opencensus-impl:0.31.0</gav>
+              </gavs>
+              <url>https://raw.githubusercontent.com/census-instrumentation/opencensus-java/v0.31.0/LICENSE</url>
+            </override>
+            <override>
               <gav>com.google.api-client:google-api-client:1.35.1</gav>
               <url>https://raw.githubusercontent.com/googleapis/google-api-java-client/v1.35.1/LICENSE</url>
             </override>
@@ -402,6 +420,23 @@
               <url>https://raw.githubusercontent.com/google/j2objc/1.3/LICENSE</url>
             </override>
             <override>
+              <gavs>
+                <gav>io.grpc:grpc-alts:1.43.2</gav>
+                <gav>io.grpc:grpc-api:1.43.2</gav>
+                <gav>io.grpc:grpc-auth:1.43.2</gav>
+                <gav>io.grpc:grpc-census:1.43.2</gav>
+                <gav>io.grpc:grpc-core:1.43.2</gav>
+                <gav>io.grpc:grpc-grpclb:1.43.2</gav>
+                <gav>io.grpc:grpc-protobuf-lite:1.43.2</gav>
+                <gav>io.grpc:grpc-protobuf:1.43.2</gav>
+                <gav>io.grpc:grpc-services:1.43.2</gav>
+                <gav>io.grpc:grpc-stub:1.43.2</gav>
+                <gav>io.grpc:grpc-xds:1.43.2</gav>
+              </gavs>
+              <url>https://raw.githubusercontent.com/grpc/grpc-java/v1.43.2/LICENSE</url>
+              <noticeUrl>https://raw.githubusercontent.com/grpc/grpc-java/v1.43.2/NOTICE.txt</noticeUrl>
+            </override>
+            <override>
               <gav>io.grpc:grpc-context:1.47.0</gav>
               <url>https://raw.githubusercontent.com/grpc/grpc-java/v1.47.0/LICENSE</url>
               <noticeUrl>https://raw.githubusercontent.com/grpc/grpc-java/v1.47.0/NOTICE.txt</noticeUrl>
@@ -422,6 +457,68 @@
               <gav>org.slf4j:slf4j-reload4j:1.7.36</gav>
               <url>https://raw.githubusercontent.com/qos-ch/slf4j/v_1.7.36/LICENSE.txt</url>
             </override>
+            <override>
+              <gavs>
+                <gav>com.google.cloud.bigdataoss:gcs-connector:hadoop3-2.2.6</gav>
+                <gav>com.google.cloud.bigdataoss:gcsio:2.2.6</gav>
+                <gav>com.google.cloud.bigdataoss:util-hadoop:hadoop3-2.2.6</gav>
+                <gav>com.google.cloud.bigdataoss:util:2.2.6</gav>
+              </gavs>
+              <url>https://raw.githubusercontent.com/GoogleCloudDataproc/hadoop-connectors/v2.2.6/LICENSE</url>
+            </override>
+            <override>
+              <gavs>
+                <gav>com.google.flogger:flogger:0.7.1</gav>
+                <gav>com.google.flogger:google-extensions:0.7.1</gav>
+                <gav>com.google.flogger:flogger-system-backend:0.7.1</gav>
+              </gavs>
+              <url>https://raw.githubusercontent.com/google/flogger/flogger-0.7.1/LICENSE</url>
+            </override>
+            <override>
+              <gavs>
+                <gav>com.google.api.grpc:proto-google-cloud-monitoring-v3:1.64.0</gav>
+                <gav>com.google.api.grpc:proto-google-cloud-storage-v2:2.2.2-alpha</gav>
+                <gav>com.google.api.grpc:grpc-google-cloud-storage-v2:2.2.2-alpha</gav>
+              </gavs>
+              <url>https://raw.githubusercontent.com/googleapis/googleapis/master/LICENSE</url>
+            </override>
+            <override>
+              <gav>com.lmax:disruptor:3.4.2</gav>
+              <url>https://raw.githubusercontent.com/LMAX-Exchange/disruptor/3.4.2/LICENCE.txt</url>
+            </override>
+            <override>
+              <gav>com.google.cloud:google-cloud-core-grpc:1.82.0</gav>
+              <url>https://raw.githubusercontent.com/googleapis/java-core/v1.82.0/LICENSE</url>
+            </override>
+            <override>
+              <gav>com.google.cloud:google-cloud-monitoring:1.82.0</gav>
+              <url>https://raw.githubusercontent.com/googleapis/java-monitoring/1.82.0/LICENSE</url>
+            </override>
+            <override>
+              <gav>com.google.api-client:google-api-client-jackson2:1.32.2</gav>
+              <url>https://raw.githubusercontent.com/googleapis/google-api-java-client/v1.32.2/LICENSE</url>
+            </override>
+            <override>
+              <gav>org.conscrypt:conscrypt-openjdk-uber:2.5.1</gav>
+              <url>https://raw.githubusercontent.com/google/conscrypt/2.5.1/LICENSE</url>
+              <noticeUrl>https://raw.githubusercontent.com/google/conscrypt/2.5.1/NOTICE</noticeUrl>
+            </override>
+            <override>
+              <gav>io.perfmark:perfmark-api:0.23.0</gav>
+              <url>https://raw.githubusercontent.com/perfmark/perfmark/v0.23.0/LICENSE</url>
+              <noticeUrl>https://raw.githubusercontent.com/perfmark/perfmark/v0.23.0/NOTICE</noticeUrl>
+            </override>
+            <override>
+              <gav>com.google.api:gax-grpc:2.7.1</gav>
+              <url>https://raw.githubusercontent.com/googleapis/gax-java/v2.7.1/LICENSE</url>
+            </override>
+            <override>
+              <gavs>
+                <gav>org.bouncycastle:bcprov-jdk15on:1.60</gav>
+                <gav>org.bouncycastle:bcpkix-jdk15on:1.60</gav>
+              </gavs>
+              <url>https://raw.githubusercontent.com/bcgit/bc-java/r1rv60/LICENSE.html</url>
+            </override>
           </overrides>
           <licenses>
             <license>
@@ -470,13 +567,25 @@
                 <aliasUrl>https://raw.githubusercontent.com/googleapis/java-iam/v1.4.1/proto-google-iam-v1/LICENSE</aliasUrl>
                 <aliasUrl>https://raw.githubusercontent.com/googleapis/java-storage/v2.9.0/LICENSE</aliasUrl>
                 <aliasUrl>http://repository.jboss.org/licenses/apache-2.0.txt</aliasUrl>
-                <aliasUrl>https://raw.githubusercontent.com/googleapis/google-http-java-client/v1.42.0/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/census-instrumentation/opencensus-java/v0.31.0/LICENSE</aliasUrl>
                 <aliasUrl>https://raw.githubusercontent.com/census-instrumentation/opencensus-java/v0.31.1/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/census-instrumentation/opencensus-proto/v0.2.0/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/google/conscrypt/2.5.1/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/google/flogger/flogger-0.7.1/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/googleapis/google-api-java-client/v1.32.2/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/googleapis/google-api-java-client/v1.35.1/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/googleapis/googleapis/master/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/googleapis/java-core/v1.82.0/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/googleapis/java-monitoring/1.82.0/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/GoogleCloudDataproc/hadoop-connectors/v2.2.6/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/LMAX-Exchange/disruptor/3.4.2/LICENCE.txt</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/perfmark/perfmark/v0.23.0/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/grpc/grpc-java/v1.43.2/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/grpc/grpc-java/v1.47.0/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/googleapis/google-http-java-client/v1.42.0/LICENSE</aliasUrl>
                 <aliasUrl>https://raw.githubusercontent.com/googleapis/google-oauth-java-client/v1.34.1/LICENSE</aliasUrl>
                 <aliasUrl>https://raw.githubusercontent.com/googleapis/java-core/v2.8.0/LICENSE</aliasUrl>
                 <aliasUrl>https://raw.githubusercontent.com/google/gson/gson-parent-2.9.0/LICENSE</aliasUrl>
-                <aliasUrl>https://raw.githubusercontent.com/googleapis/google-api-java-client/v1.35.1/LICENSE</aliasUrl>
-                <aliasUrl>https://raw.githubusercontent.com/grpc/grpc-java/v1.47.0/LICENSE</aliasUrl>
               </aliasUrls>
               <metric>1</metric>
             </license>
@@ -864,7 +973,7 @@
           <plugin>
             <artifactId>jdeb</artifactId>
             <groupId>org.vafer</groupId>
-            <version>1.5</version>
+            <version>1.8</version>
             <executions>
               <execution>
                 <phase>package</phase>
@@ -874,26 +983,36 @@
                 <configuration>
                   <dataSet>
                     <data>
-                      <src>${project.build.directory}/${project.build.finalName}-binary-assembly/apache-asterixdb-${project.version}/</src>
-                      <excludes>bin/**</excludes>
+                      <src>${project.build.directory}/${project.build.finalName}-binary-assembly/apache-asterixdb-${project.version}</src>
                       <type>directory</type>
                       <mapper>
                         <type>perm</type>
-                        <prefix>/opt/apache-asterixdb-${project.version}/</prefix>
-                        <user>asterixdb</user>
-                        <group>asterixdb</group>
+                        <prefix>/opt/apache-asterixdb/</prefix>
+                        <user>root</user>
+                        <group>root</group>
+                        <filemode>755</filemode>
+                      </mapper>
+                    </data>
+                    <data>
+                      <type>file</type>
+                      <src>src/deb/systemd/cc.conf</src>
+                      <mapper>
+                        <prefix>/opt/apache-asterixdb/</prefix>
+                        <type>perm</type>
+                        <user>root</user>
+                        <group>root</group>
                         <filemode>644</filemode>
                       </mapper>
                     </data>
                     <data>
-                      <src>${project.build.directory}/${project.build.finalName}-binary-assembly/apache-asterixdb-${project.version}/bin</src>
-                      <type>directory</type>
+                      <type>file</type>
+                      <src>src/deb/udf_listener.py</src>
                       <mapper>
+                        <prefix>/opt/apache-asterixdb/bin</prefix>
                         <type>perm</type>
-                        <prefix>/opt/apache-asterixdb-${project.version}/bin</prefix>
-                        <user>asterixdb</user>
-                        <group>asterixdb</group>
-                        <filemode>754</filemode>
+                        <user>root</user>
+                        <group>root</group>
+                        <filemode>555</filemode>
                       </mapper>
                     </data>
                     <data>
@@ -916,6 +1035,39 @@
                         <group>root</group>
                       </mapper>
                     </data>
+                    <data>
+                      <type>file</type>
+                      <src>src/deb/systemd/pyudf.socket</src>
+                      <mapper>
+                        <prefix>/lib/systemd/system</prefix>
+                        <type>perm</type>
+                        <user>root</user>
+                        <group>root</group>
+                      </mapper>
+                    </data>
+                    <data>
+                      <type>file</type>
+                      <src>src/deb/systemd/pyudf@.service</src>
+                      <mapper>
+                        <prefix>/lib/systemd/system</prefix>
+                        <type>perm</type>
+                        <user>root</user>
+                        <group>root</group>
+                      </mapper>
+                    </data>
+                    <data>
+                      <type>template</type>
+                      <paths>
+                          <path>/opt/apache-asterixdb/logs</path>
+                          <path>/opt/apache-asterixdb/data</path>
+                      </paths>
+                      <mapper>
+                        <type>perm</type>
+                        <user>asterixdb</user>
+                        <group>asterixdb</group>
+                        <filemode>750</filemode>
+                      </mapper>
+                    </data>
                   </dataSet>
                 </configuration>
               </execution>
diff --git a/asterixdb/asterix-server/src/deb/control/control b/asterixdb/asterix-server/src/deb/control/control
index 1f6c213..77bbd1d 100644
--- a/asterixdb/asterix-server/src/deb/control/control
+++ b/asterixdb/asterix-server/src/deb/control/control
@@ -17,8 +17,7 @@
 Section: databases
 Priority: extra
 Architecture: all
-Depends: jdk (>= 1.8)
+Depends: java17-runtime-headless
 Maintainer: Ian Maxon <ian@maxons.email>
 Description: Apache AsterixDB - a scalable, open source Big Data Management System (BDMS)
-Distribution: development
-Depends: default-jre | java8-runtime
+Distribution: development
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/deb/control/postinst b/asterixdb/asterix-server/src/deb/control/postinst
index 896ca28..fe5c912 100644
--- a/asterixdb/asterix-server/src/deb/control/postinst
+++ b/asterixdb/asterix-server/src/deb/control/postinst
@@ -13,5 +13,4 @@
 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 # See the License for the specific language governing permissions and
 # limitations under the License.
-adduser --system --group --quiet --home /opt/apache-asterixdb/ \
---no-create-home --disabled-login --force-badname asterixdb
+chmod -R 755 /opt/apache-asterixdb/
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/deb/control/preinst b/asterixdb/asterix-server/src/deb/control/preinst
index 4509c90..8d14847 100644
--- a/asterixdb/asterix-server/src/deb/control/preinst
+++ b/asterixdb/asterix-server/src/deb/control/preinst
@@ -13,3 +13,7 @@
 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 # See the License for the specific language governing permissions and
 # limitations under the License.
+adduser --system --group --quiet --home /opt/apache-asterixdb/ \
+--no-create-home --disabled-login --force-badname asterixdb
+adduser --system --group --quiet --home /opt/apache-asterixdb/ \
+--no-create-home --disabled-login --force-badname asterixdb-udf
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/deb/systemd/asterix-cc.service b/asterixdb/asterix-server/src/deb/systemd/asterix-cc.service
index 9711fba..2a52e2d 100644
--- a/asterixdb/asterix-server/src/deb/systemd/asterix-cc.service
+++ b/asterixdb/asterix-server/src/deb/systemd/asterix-cc.service
@@ -19,8 +19,9 @@
 [Service]
 Type=simple
 User=asterixdb
-ExecStart=/opt/apache-asterixdb/bin/asterixcc --config-file /opt/apache-asterixdb/cc.conf
+ExecStart=/opt/apache-asterixdb/bin/asterixcc -config-file "/opt/apache-asterixdb/cc.conf"
 Restart=on-abort
+WorkingDirectory=/opt/apache-asterixdb
 
 [Install]
 WantedBy=multi-user.target
diff --git a/asterixdb/asterix-server/src/deb/systemd/asterix-nc.service b/asterixdb/asterix-server/src/deb/systemd/asterix-nc.service
index bfe6296..e09d8e8 100644
--- a/asterixdb/asterix-server/src/deb/systemd/asterix-nc.service
+++ b/asterixdb/asterix-server/src/deb/systemd/asterix-nc.service
@@ -21,6 +21,7 @@
 User=asterixdb
 ExecStart=/opt/apache-asterixdb/bin/asterixncservice
 Restart=on-abort
+WorkingDirectory=/opt/apache-asterixdb
 
 [Install]
 WantedBy=multi-user.target
diff --git a/asterixdb/asterix-server/src/deb/systemd/cc.conf b/asterixdb/asterix-server/src/deb/systemd/cc.conf
new file mode 100644
index 0000000..0af967a
--- /dev/null
+++ b/asterixdb/asterix-server/src/deb/systemd/cc.conf
@@ -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.
+
+[nc/asterix_nc1]
+txn.log.dir=/opt/apache-asterixdb/data/txnlog
+core.dump.dir=/opt/apache-asterixdb/logs/coredump
+iodevices=/opt/apache-asterixdb/data/
+nc.api.port=19004
+
+[nc]
+address=127.0.0.1
+command=asterixnc
+
+[cc]
+address = 127.0.0.1
+
+[common]
+log.level = INFO
+log.dir = /opt/apache-asterixdb/logs/
diff --git a/asterixdb/asterix-server/src/deb/systemd/pyudf.socket b/asterixdb/asterix-server/src/deb/systemd/pyudf.socket
new file mode 100644
index 0000000..4e731db
--- /dev/null
+++ b/asterixdb/asterix-server/src/deb/systemd/pyudf.socket
@@ -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.
+[Unit]
+Description=AsterixDB UDF Domain Socket
+PartOf=asterixdb_udf.service
+
+[Socket]
+ListenStream=/tmp/pyudf.socket
+SocketMode=0660
+SocketUser=asterixdb-udf
+SocketGroup=asterixdb
+Accept=true
+DeferAcceptSec=1
+
+[Install]
+WantedBy=sockets.target
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/deb/systemd/pyudf@.service b/asterixdb/asterix-server/src/deb/systemd/pyudf@.service
new file mode 100644
index 0000000..9856142
--- /dev/null
+++ b/asterixdb/asterix-server/src/deb/systemd/pyudf@.service
@@ -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.
+[Unit]
+Description=AsterixDB UDF Executor Service
+After=network.target pyudf.socket
+Requires=pyudf.socket
+
+[Service]
+User=asterixdb-udf
+Type=simple
+ExecStart=/usr/bin/python3 /opt/apache-asterixdb/bin/udf_listener.py
+TimeoutStopSec=5
+StandardError=journal
+StandardError=journal
+
+[Install]
+WantedBy=default.target
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/deb/udf_listener.py b/asterixdb/asterix-server/src/deb/udf_listener.py
new file mode 100644
index 0000000..03874b2
--- /dev/null
+++ b/asterixdb/asterix-server/src/deb/udf_listener.py
@@ -0,0 +1,283 @@
+#!/usr/bin/env python3
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import sys
+from systemd.daemon import listen_fds
+from os import chdir
+from os import getcwd
+from os import getpid
+from struct import *
+import signal
+import msgpack
+import socket
+import traceback
+from importlib import import_module
+from pathlib import Path
+from enum import IntEnum
+from io import BytesIO
+
+
+PROTO_VERSION = 1
+HEADER_SZ = 8 + 8 + 1
+REAL_HEADER_SZ = 4 + 8 + 8 + 1
+FRAMESZ = 32768
+
+
+class MessageType(IntEnum):
+    HELO = 0
+    QUIT = 1
+    INIT = 2
+    INIT_RSP = 3
+    CALL = 4
+    CALL_RSP = 5
+    ERROR = 6
+
+
+class MessageFlags(IntEnum):
+    NORMAL = 0
+    INITIAL_REQ = 1
+    INITIAL_ACK = 2
+    ERROR = 3
+
+
+class Wrapper(object):
+    wrapped_module = None
+    wrapped_class = None
+    wrapped_fn = None
+    sz = None
+    mid = None
+    rmid = None
+    flag = None
+    resp = None
+    unpacked_msg = None
+    msg_type = None
+    packer = msgpack.Packer(autoreset=False, use_bin_type=False)
+    unpacker = msgpack.Unpacker(raw=False)
+    response_buf = BytesIO()
+    stdin_buf = BytesIO()
+    wrapped_fns = {}
+    alive = True
+    readbuf = bytearray(FRAMESZ)
+    readview = memoryview(readbuf)
+
+
+    def init(self, module_name, class_name, fn_name):
+        self.wrapped_module = import_module(module_name)
+        # do not allow modules to be called that are not part of the uploaded module
+        wrapped_fn = None
+        if not self.check_module_path(self.wrapped_module):
+            self.wrapped_module = None
+            raise ImportError("Module was not found in library")
+        if class_name is not None:
+            self.wrapped_class = getattr(
+                import_module(module_name), class_name)()
+        if self.wrapped_class is not None:
+            wrapped_fn = getattr(self.wrapped_class, fn_name)
+        else:
+            wrapped_fn = getattr(import_module(module_name), fn_name)
+        if wrapped_fn is None:
+            raise ImportError(
+                "Could not find class or function in specified module")
+        self.wrapped_fns[self.mid] = wrapped_fn
+
+    def next_tuple(self, *args, key=None):
+        return self.wrapped_fns[key](*args)
+
+    def check_module_path(self, module):
+        cwd = Path('.').resolve()
+        module_path = Path(module.__file__).resolve()
+        return cwd in module_path.parents
+        return True
+
+    def read_header(self, readbuf):
+        self.sz, self.mid, self.rmid, self.flag = unpack(
+            "!iqqb", readbuf[0:REAL_HEADER_SZ])
+        return True
+
+    def write_header(self, response_buf, dlen):
+        total_len = dlen + HEADER_SZ
+        header = pack("!iqqb", total_len, int(-1), int(self.rmid), self.flag)
+        self.response_buf.write(header)
+        return total_len + 4
+
+    def get_ver_hlen(self, hlen):
+        return hlen + (PROTO_VERSION << 4)
+
+    def get_hlen(self):
+        return self.ver_hlen - (PROTO_VERSION << 4)
+
+    def init_remote_ipc(self):
+        self.response_buf.seek(0)
+        self.flag = MessageFlags.INITIAL_REQ
+        dlen = len(self.unpacked_msg[1])
+        resp_len = self.write_header(self.response_buf, dlen)
+        self.response_buf.write(self.unpacked_msg[1])
+        self.resp = self.response_buf.getbuffer()[0:resp_len]
+        self.send_msg()
+        self.packer.reset()
+
+    def cd(self, basedir):
+        chdir(basedir + "/site-packages")
+        sys.path.insert(0,getcwd())
+
+    def helo(self):
+        # need to ack the connection back before sending actual HELO
+        #   self.init_remote_ipc()
+        self.cd(self.unpacked_msg[1][1])
+        self.flag = MessageFlags.NORMAL
+        self.response_buf.seek(0)
+        self.packer.pack(int(MessageType.HELO))
+        self.packer.pack(int(getpid()))
+        dlen = len(self.packer.bytes())  # tag(1) + body(4)
+        resp_len = self.write_header(self.response_buf, dlen)
+        self.response_buf.write(self.packer.bytes())
+        self.resp = self.response_buf.getbuffer()[0:resp_len]
+        self.send_msg()
+        self.packer.reset()
+        return True
+
+    def handle_init(self):
+        self.flag = MessageFlags.NORMAL
+        self.response_buf.seek(0)
+        args = self.unpacked_msg[1]
+        module = args[0]
+        if len(args) == 3:
+            clazz = args[1]
+            fn = args[2]
+        else:
+            clazz = None
+            fn = args[1]
+        self.init(module, clazz, fn)
+        self.packer.pack(int(MessageType.INIT_RSP))
+        dlen = 1  # just the tag.
+        resp_len = self.write_header(self.response_buf, dlen)
+        self.response_buf.write(self.packer.bytes())
+        self.resp = self.response_buf.getbuffer()[0:resp_len]
+        self.send_msg()
+        self.packer.reset()
+        return True
+
+    def quit(self):
+        self.alive = False
+        return True
+
+    def handle_call(self):
+        self.flag = MessageFlags.NORMAL
+        result = ([], [])
+        if len(self.unpacked_msg) > 1:
+            args = self.unpacked_msg[1]
+            if args is not None:
+                for arg in args:
+                    try:
+                        result[0].append(self.next_tuple(*arg, key=self.mid))
+                    except BaseException as e:
+                        result[1].append(traceback.format_exc())
+        self.packer.reset()
+        self.response_buf.seek(0)
+        body = msgpack.packb(result)
+        dlen = len(body) + 1  # 1 for tag
+        resp_len = self.write_header(self.response_buf, dlen)
+        self.packer.pack(int(MessageType.CALL_RSP))
+        self.response_buf.write(self.packer.bytes())
+        self.response_buf.write(body)
+        self.resp = self.response_buf.getbuffer()[0:resp_len]
+        self.send_msg()
+        self.packer.reset()
+        return True
+
+    def handle_error(self, e):
+        self.flag = MessageFlags.NORMAL
+        self.packer.reset()
+        self.response_buf.seek(0)
+        body = msgpack.packb(str(e))
+        dlen = len(body) + 1  # 1 for tag
+        resp_len = self.write_header(self.response_buf, dlen)
+        self.packer.pack(int(MessageType.ERROR))
+        self.response_buf.write(self.packer.bytes())
+        self.response_buf.write(body)
+        self.resp = self.response_buf.getbuffer()[0:resp_len]
+        self.send_msg()
+        self.packer.reset()
+        self.alive = False
+        return True
+
+    type_handler = {
+        MessageType.HELO: helo,
+        MessageType.QUIT: quit,
+        MessageType.INIT: handle_init,
+        MessageType.CALL: handle_call
+    }
+
+    def connect_sock(self):
+        self.sock = socket.fromfd(listen_fds()[0], socket.AF_UNIX, socket.SOCK_STREAM)
+
+    def disconnect_sock(self, *args):
+        self.sock.shutdown(socket.SHUT_RDWR)
+        self.sock.close()
+
+    def recv_msg(self):
+        while self.alive:
+            pos = self.sock.recv_into(self.readbuf)
+            if pos <= 0:
+                self.alive = False
+                return
+            try:
+                while pos < REAL_HEADER_SZ:
+                    read = self.sock.recv_into(self.readview[pos:])
+                    if read <= 0:
+                        self.alive = False
+                        return
+                    pos += read
+                self.read_header(self.readview)
+                while pos < self.sz and len(self.readbuf) - pos > 0:
+                    read = self.sock.recv_into(self.readview[pos:])
+                    if read <= 0:
+                        self.alive = False
+                        return
+                    pos += read
+                while pos < self.sz:
+                    vszchunk = self.sock.recv(4096)
+                    if len(vszchunk) == 0:
+                        self.alive = False
+                        return
+                    self.readview.release()
+                    self.readbuf.extend(vszchunk)
+                    self.readview = memoryview(self.readbuf)
+                    pos += len(vszchunk)
+                self.unpacker.feed(self.readview[REAL_HEADER_SZ:self.sz])
+                self.unpacked_msg = list(self.unpacker)
+                self.msg_type = MessageType(self.unpacked_msg[0])
+                self.type_handler[self.msg_type](self)
+            except BaseException as e:
+                self.handle_error(''.join(traceback.format_exc()))
+
+    def send_msg(self):
+        self.sock.sendall(self.resp)
+        self.resp = None
+        return
+
+    def recv_loop(self):
+        while self.alive:
+            self.recv_msg()
+        self.disconnect_sock()
+
+
+wrap = Wrapper()
+wrap.connect_sock()
+signal.signal(signal.SIGTERM, wrap.disconnect_sock)
+wrap.recv_loop()
diff --git a/asterixdb/asterix-server/src/test/resources/NCServiceExecutionIT/cc.conf b/asterixdb/asterix-server/src/test/resources/NCServiceExecutionIT/cc.conf
index 1c6d574..2523b6f 100644
--- a/asterixdb/asterix-server/src/test/resources/NCServiceExecutionIT/cc.conf
+++ b/asterixdb/asterix-server/src/test/resources/NCServiceExecutionIT/cc.conf
@@ -46,5 +46,7 @@
 heartbeat.max.misses=25
 
 [common]
+compiler.cbo=false
+compiler.cbotest=true
 log.dir = ../asterix-server/target/NCServiceExecutionIT
 log.level = INFO
\ No newline at end of file
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index d48eb8a..561bc03 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -94,6 +94,7 @@
     <azuredatalakejavasdk.version>12.7.2</azuredatalakejavasdk.version>
     <gcsjavasdk.version>2.9.0</gcsjavasdk.version>
     <hadoop-azuresdk.version>8.6.6</hadoop-azuresdk.version>
+    <hadoop-gcs.version>hadoop3-2.2.6</hadoop-gcs.version>
 
     <implementation.title>Apache AsterixDB - ${project.name}</implementation.title>
     <implementation.url>https://asterixdb.apache.org/</implementation.url>
@@ -930,7 +931,7 @@
     <module>asterix-test-framework</module>
     <module>asterix-maven-plugins</module>
     <module>asterix-server</module>
-    <module>asterix-docker</module>
+    <module>asterix-podman</module>
     <module>asterix-doc</module>
     <module>asterix-fuzzyjoin</module>
     <module>asterix-replication</module>
@@ -2002,6 +2003,19 @@
         </exclusions>
       </dependency>
       <!-- Hadoop Azure end -->
+      <!-- Hadoop GCS start -->
+      <dependency>
+        <groupId>com.google.cloud.bigdataoss</groupId>
+        <artifactId>gcs-connector</artifactId>
+        <version>${hadoop-gcs.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.checkerframework</groupId>
+            <artifactId>checker-compat-qual</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <!-- Hadoop GCS end -->
 
       <!-- TODO(htowaileb): removed from hadoop transitively and added separately to avoid CVEs, can
            be removed once upgraded to hadoop 3.3.4 as it addresses the CVEs -->
diff --git a/asterixdb/src/main/appended-resources/supplemental-models.xml b/asterixdb/src/main/appended-resources/supplemental-models.xml
index 5c8bd8a..7cdbe35 100644
--- a/asterixdb/src/main/appended-resources/supplemental-models.xml
+++ b/asterixdb/src/main/appended-resources/supplemental-models.xml
@@ -1173,6 +1173,31 @@
     </project>
   </supplement>
 
+  <!-- io.opencensus uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>io.opencensus</groupId>
+      <artifactId>opencensus-proto</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.2.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.2.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.2.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.android:annotations has ASLv2 <license> in pom -->
+  <supplement>
+    <project>
+      <groupId>com.google.android</groupId>
+      <artifactId>annotations</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>4.1.1.4</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>4.1.1.4</license.ignoreMissingEmbeddedNotice>
+      </properties>
+    </project>
+  </supplement>
+
   <!-- io.opencensus uses non-fixed ALv2 with no NOTICE file -->
   <supplement>
     <project>
@@ -1186,6 +1211,32 @@
     </project>
   </supplement>
 
+  <!-- io.opencensus uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>io.opencensus</groupId>
+      <artifactId>opencensus-contrib-exemplar-util</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.opencensus uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>io.opencensus</groupId>
+      <artifactId>opencensus-contrib-grpc-metrics</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
   <!-- io.opencensus uses non-fixed ALv2 with no NOTICE file -->
   <supplement>
     <project>
@@ -1199,6 +1250,316 @@
     </project>
   </supplement>
 
+  <!-- io.opencensus uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>io.opencensus</groupId>
+      <artifactId>opencensus-contrib-resource-util</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.opencensus uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>io.opencensus</groupId>
+      <artifactId>opencensus-exporter-metrics-util</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.opencensus uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>io.opencensus</groupId>
+      <artifactId>opencensus-exporter-stats-stackdriver</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.opencensus uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>io.opencensus</groupId>
+      <artifactId>opencensus-impl-core</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.opencensus uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>io.opencensus</groupId>
+      <artifactId>opencensus-impl</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.cloud.bigdataoss uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.cloud.bigdataoss</groupId>
+      <artifactId>gcs-connector</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>hadoop3-2.2.6</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>hadoop3-2.2.6</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>hadoop3-2.2.6</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.cloud.bigdataoss uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.cloud.bigdataoss</groupId>
+      <artifactId>gcsio</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>2.2.6</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>2.2.6</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>2.2.6</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.cloud.bigdataoss uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.cloud.bigdataoss</groupId>
+      <artifactId>util-hadoop</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>hadoop3-2.2.6</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>hadoop3-2.2.6</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>hadoop3-2.2.6</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.cloud.bigdataoss uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.cloud.bigdataoss</groupId>
+      <artifactId>util</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>2.2.6</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>2.2.6</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>2.2.6</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.flogger uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.flogger</groupId>
+      <artifactId>flogger</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.7.1</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.7.1</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.7.1</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.flogger uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.flogger</groupId>
+      <artifactId>google-extensions</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.7.1</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.7.1</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.7.1</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.flogger uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.flogger</groupId>
+      <artifactId>flogger-system-backend</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.7.1</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.7.1</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.7.1</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.api.grpc uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.api.grpc</groupId>
+      <artifactId>proto-google-cloud-monitoring-v3</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.64.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.64.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.64.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.api.grpc uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.api.grpc</groupId>
+      <artifactId>grpc-google-cloud-storage-v2</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>2.2.2-alpha</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>2.2.2-alpha</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>2.2.2-alpha</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.api.grpc uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.api.grpc</groupId>
+      <artifactId>proto-google-cloud-storage-v2</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>2.2.2-alpha</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>2.2.2-alpha</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>2.2.2-alpha</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.lmax uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.lmax</groupId>
+      <artifactId>disruptor</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>3.4.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>3.4.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>3.4.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.cloud uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.cloud</groupId>
+      <artifactId>google-cloud-core-grpc</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.82.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.82.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.82.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.cloud uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.cloud</groupId>
+      <artifactId>google-cloud-monitoring</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.82.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.82.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.82.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.api-client uses ALv2 LICENSE and has a NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.api-client</groupId>
+      <artifactId>google-api-client-jackson2</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.32.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.32.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.32.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- org.conscrypt uses ALv2 LICENSE and has a NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>org.conscrypt</groupId>
+      <artifactId>conscrypt-openjdk-uber</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>2.5.1</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>2.5.1</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>2.5.1</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.perfmark uses ALv2 license -->
+  <supplement>
+    <project>
+      <groupId>io.perfmark</groupId>
+      <artifactId>perfmark-api</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.23.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.23.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.23.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- org.codehaus.mojo:animal-sniffer-annotations uses MIT license -->
+  <supplement>
+    <project>
+      <groupId>org.codehaus.mojo</groupId>
+      <artifactId>animal-sniffer-annotations</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.19</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.19</license.ignoreMissingEmbeddedNotice>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.apis:google-api-services-iamcredentials has embedded ASLv2 in pom.xml -->
+  <supplement>
+    <project>
+      <groupId>com.google.apis</groupId>
+      <artifactId>google-api-services-iamcredentials</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>v1-rev20210326-1.32.1</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>v1-rev20210326-1.32.1</license.ignoreMissingEmbeddedNotice>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.apis:google-api-services-iamcredentials has embedded ASLv2 in pom.xml -->
+  <supplement>
+    <project>
+      <groupId>com.google.api</groupId>
+      <artifactId>gax-grpc</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>2.7.1</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>2.7.1</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>2.7.1</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
   <!-- com.google.cloud:google-cloud-core is non-fixed ALv2 with no NOTICE file -->
   <supplement>
     <project>
@@ -1251,7 +1612,59 @@
     </project>
   </supplement>
 
-  <!-- io.grpc:grpc-context uses non-fixed ALv2 and has no NOTICE -->
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-alts</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-api</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-auth</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-census</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
   <supplement>
     <project>
       <groupId>io.grpc</groupId>
@@ -1264,6 +1677,97 @@
     </project>
   </supplement>
 
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-core</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-grpclb</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-protobuf-lite</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-protobuf</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-services</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-stub</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-xds</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
   <!-- com.google.protobuf has no NOTICE file -->
   <supplement>
     <project>
@@ -1400,7 +1904,7 @@
     </project>
   </supplement>
 
-  <!-- org.checkerframework:checker-compat-qual has no NOTICE file -->
+  <!-- org.checkerframework:checker-qual has no NOTICE file -->
   <supplement>
     <project>
       <groupId>org.checkerframework</groupId>
@@ -1737,4 +2241,28 @@
       </properties>
     </project>
   </supplement>
+
+  <supplement>
+    <project>
+      <groupId>org.bouncycastle</groupId>
+      <artifactId>bcpkix-jdk15on</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedNotice>1.60</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreMissingEmbeddedLicense>1.60</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreLicenseOverride>1.60</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <supplement>
+    <project>
+      <groupId>org.bouncycastle</groupId>
+      <artifactId>bcprov-jdk15on</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedNotice>1.60</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreMissingEmbeddedLicense>1.60</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreLicenseOverride>1.60</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
 </supplementalDataModels>
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_bcgit_bc-java_r1rv60_LICENSE.html.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_bcgit_bc-java_r1rv60_LICENSE.html.txt
new file mode 100644
index 0000000..b8f46c5
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_bcgit_bc-java_r1rv60_LICENSE.html.txt
@@ -0,0 +1,17 @@
+Copyright (c) 2000-2018 The Legion of the Bouncy Castle Inc. (http://www.bouncycastle.org)
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software
+and associated documentation files (the "Software"), to deal in the Software without restriction,
+including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense,
+and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so,
+subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial
+portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED,
+INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR
+PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
+OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
+DEALINGS IN THE SOFTWARE.
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_conscrypt_2.5.1_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_conscrypt_2.5.1_NOTICE.txt
new file mode 100644
index 0000000..80715a6
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_conscrypt_2.5.1_NOTICE.txt
@@ -0,0 +1,30 @@
+Copyright 2016 The Android Open Source Project
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-----------------------------------------------------------------------
+This product contains a modified portion of `Netty`, a configurable network
+stack in Java, which can be obtained at:
+
+  * LICENSE:
+    * licenses/LICENSE.netty.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://netty.io/
+
+This product contains a modified portion of `Apache Harmony`, modular Java runtime,
+which can be obtained at:
+
+  * LICENSE:
+    * licenses/LICENSE.harmony.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://harmony.apache.org/
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gax-java_v2.7.1_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gax-java_v2.7.1_LICENSE.txt
new file mode 100644
index 0000000..6874140
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gax-java_v2.7.1_LICENSE.txt
@@ -0,0 +1,27 @@
+Copyright 2016, Google Inc. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+    * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.43.2_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.43.2_NOTICE.txt
new file mode 100644
index 0000000..c5d3ec2
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.43.2_NOTICE.txt
@@ -0,0 +1,62 @@
+Copyright 2014 The gRPC Authors
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-----------------------------------------------------------------------
+
+This product contains a modified portion of 'OkHttp', an open source
+HTTP & SPDY client for Android and Java applications, which can be obtained
+at:
+
+  * LICENSE:
+    * okhttp/third_party/okhttp/LICENSE (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/square/okhttp
+  * LOCATION_IN_GRPC:
+    * okhttp/third_party/okhttp
+
+This product contains a modified portion of 'Envoy', an open source
+cloud-native high-performance edge/middle/service proxy, which can be
+obtained at:
+
+  * LICENSE:
+    * xds/third_party/envoy/LICENSE (Apache License 2.0)
+  * NOTICE:
+    * xds/third_party/envoy/NOTICE
+  * HOMEPAGE:
+    * https://www.envoyproxy.io
+  * LOCATION_IN_GRPC:
+    * xds/third_party/envoy
+
+This product contains a modified portion of 'protoc-gen-validate (PGV)',
+an open source protoc plugin to generate polyglot message validators,
+which can be obtained at:
+
+  * LICENSE:
+    * xds/third_party/protoc-gen-validate/LICENSE (Apache License 2.0)
+  * NOTICE:
+      * xds/third_party/protoc-gen-validate/NOTICE
+  * HOMEPAGE:
+    * https://github.com/envoyproxy/protoc-gen-validate
+  * LOCATION_IN_GRPC:
+    * xds/third_party/protoc-gen-validate
+
+This product contains a modified portion of 'udpa',
+an open source universal data plane API, which can be obtained at:
+
+  * LICENSE:
+    * xds/third_party/udpa/LICENSE (Apache License 2.0)
+  * HOMEPAGE:
+    * https://github.com/cncf/udpa
+  * LOCATION_IN_GRPC:
+    * xds/third_party/udpa
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_perfmark_perfmark_v0.23.0_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_perfmark_perfmark_v0.23.0_NOTICE.txt
new file mode 100644
index 0000000..3ed2d12
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_perfmark_perfmark_v0.23.0_NOTICE.txt
@@ -0,0 +1,32 @@
+
+Copyright 2019 Google LLC
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-----------------------------------------------------------------------
+
+This product contains a modified portion of 'Catapult', an open source
+Trace Event viewer for Chome, Linux, and Android applications, which can
+be obtained at:
+
+  * LICENSE:
+    * traceviewer/src/main/resources/io/perfmark/traceviewer/third_party/catapult/LICENSE (New BSD License)
+  * HOMEPAGE:
+    * https://github.com/catapult-project/catapult
+
+This product contains a modified portion of 'Polymer', a library for Web
+Components, which can be obtained at:
+  * LICENSE:
+    * traceviewer/src/main/resources/io/perfmark/traceviewer/third_party/polymer/LICENSE (New BSD License)
+  * HOMEPAGE:
+    * https://github.com/Polymer/polymer
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_typetools_checker-framework_checker-framework-2.5.3_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_typetools_checker-framework_checker-framework-2.5.3_LICENSE.txt
new file mode 100644
index 0000000..fe8c705
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_typetools_checker-framework_checker-framework-2.5.3_LICENSE.txt
@@ -0,0 +1,408 @@
+Most of the Checker Framework is licensed under the GNU General Public
+License, version 2 (GPL2), with the classpath exception.  The text of this
+license appears below.  This is the same license used for OpenJDK.
+
+A few parts of the Checker Framework have more permissive licenses.
+
+ * The annotations are licensed under the MIT License.  (The text of this
+   license appears below.)  More specifically, all the parts of the Checker
+   Framework that you might want to include with your own program use the
+   MIT License.  This is the checker-qual.jar file and all the files that
+   appear in it:  every file in a qual/ directory, plus utility files such
+   as NullnessUtil.java, RegexUtil.java, SignednessUtil.java, etc.
+   In addition, the cleanroom implementations of third-party annotations,
+   which the Checker Framework recognizes as aliases for its own
+   annotations, are licensed under the MIT License.
+
+Some external libraries that are included with the Checker Framework have
+different licenses.
+
+ * javaparser is dual licensed under the LGPL or the Apache license -- you
+   may use it under whichever one you want.  (The javaparser source code
+   contains a file with the text of the GPL, but it is not clear why, since
+   javaparser does not use the GPL.)  See file stubparser/LICENSE
+   and the source code of all its files.
+
+ * JUnit is licensed under the Common Public License v1.0 (see
+   http://www.junit.org/license), with parts (Hamcrest) licensed under the
+   BSD License (see http://hamcrest.org/JavaHamcrest/).
+
+ * plume-lib is licensed under the MIT License.
+
+The Checker Framework includes annotations for the JDK in directory
+checker/jdk/, and for some other libraries.  Each annotated library uses
+the same license as the unannotated version of the library.
+
+===========================================================================
+
+The GNU General Public License (GPL)
+
+Version 2, June 1991
+
+Copyright (C) 1989, 1991 Free Software Foundation, Inc.
+59 Temple Place, Suite 330, Boston, MA 02111-1307 USA
+
+Everyone is permitted to copy and distribute verbatim copies of this license
+document, but changing it is not allowed.
+
+Preamble
+
+The licenses for most software are designed to take away your freedom to share
+and change it.  By contrast, the GNU General Public License is intended to
+guarantee your freedom to share and change free software--to make sure the
+software is free for all its users.  This General Public License applies to
+most of the Free Software Foundation's software and to any other program whose
+authors commit to using it.  (Some other Free Software Foundation software is
+covered by the GNU Library General Public License instead.) You can apply it to
+your programs, too.
+
+When we speak of free software, we are referring to freedom, not price.  Our
+General Public Licenses are designed to make sure that you have the freedom to
+distribute copies of free software (and charge for this service if you wish),
+that you receive source code or can get it if you want it, that you can change
+the software or use pieces of it in new free programs; and that you know you
+can do these things.
+
+To protect your rights, we need to make restrictions that forbid anyone to deny
+you these rights or to ask you to surrender the rights.  These restrictions
+translate to certain responsibilities for you if you distribute copies of the
+software, or if you modify it.
+
+For example, if you distribute copies of such a program, whether gratis or for
+a fee, you must give the recipients all the rights that you have.  You must
+make sure that they, too, receive or can get the source code.  And you must
+show them these terms so they know their rights.
+
+We protect your rights with two steps: (1) copyright the software, and (2)
+offer you this license which gives you legal permission to copy, distribute
+and/or modify the software.
+
+Also, for each author's protection and ours, we want to make certain that
+everyone understands that there is no warranty for this free software.  If the
+software is modified by someone else and passed on, we want its recipients to
+know that what they have is not the original, so that any problems introduced
+by others will not reflect on the original authors' reputations.
+
+Finally, any free program is threatened constantly by software patents.  We
+wish to avoid the danger that redistributors of a free program will
+individually obtain patent licenses, in effect making the program proprietary.
+To prevent this, we have made it clear that any patent must be licensed for
+everyone's free use or not licensed at all.
+
+The precise terms and conditions for copying, distribution and modification
+follow.
+
+TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION
+
+0. This License applies to any program or other work which contains a notice
+placed by the copyright holder saying it may be distributed under the terms of
+this General Public License.  The "Program", below, refers to any such program
+or work, and a "work based on the Program" means either the Program or any
+derivative work under copyright law: that is to say, a work containing the
+Program or a portion of it, either verbatim or with modifications and/or
+translated into another language.  (Hereinafter, translation is included
+without limitation in the term "modification".) Each licensee is addressed as
+"you".
+
+Activities other than copying, distribution and modification are not covered by
+this License; they are outside its scope.  The act of running the Program is
+not restricted, and the output from the Program is covered only if its contents
+constitute a work based on the Program (independent of having been made by
+running the Program).  Whether that is true depends on what the Program does.
+
+1. You may copy and distribute verbatim copies of the Program's source code as
+you receive it, in any medium, provided that you conspicuously and
+appropriately publish on each copy an appropriate copyright notice and
+disclaimer of warranty; keep intact all the notices that refer to this License
+and to the absence of any warranty; and give any other recipients of the
+Program a copy of this License along with the Program.
+
+You may charge a fee for the physical act of transferring a copy, and you may
+at your option offer warranty protection in exchange for a fee.
+
+2. You may modify your copy or copies of the Program or any portion of it, thus
+forming a work based on the Program, and copy and distribute such modifications
+or work under the terms of Section 1 above, provided that you also meet all of
+these conditions:
+
+    a) You must cause the modified files to carry prominent notices stating
+    that you changed the files and the date of any change.
+
+    b) You must cause any work that you distribute or publish, that in whole or
+    in part contains or is derived from the Program or any part thereof, to be
+    licensed as a whole at no charge to all third parties under the terms of
+    this License.
+
+    c) If the modified program normally reads commands interactively when run,
+    you must cause it, when started running for such interactive use in the
+    most ordinary way, to print or display an announcement including an
+    appropriate copyright notice and a notice that there is no warranty (or
+    else, saying that you provide a warranty) and that users may redistribute
+    the program under these conditions, and telling the user how to view a copy
+    of this License.  (Exception: if the Program itself is interactive but does
+    not normally print such an announcement, your work based on the Program is
+    not required to print an announcement.)
+
+These requirements apply to the modified work as a whole.  If identifiable
+sections of that work are not derived from the Program, and can be reasonably
+considered independent and separate works in themselves, then this License, and
+its terms, do not apply to those sections when you distribute them as separate
+works.  But when you distribute the same sections as part of a whole which is a
+work based on the Program, the distribution of the whole must be on the terms
+of this License, whose permissions for other licensees extend to the entire
+whole, and thus to each and every part regardless of who wrote it.
+
+Thus, it is not the intent of this section to claim rights or contest your
+rights to work written entirely by you; rather, the intent is to exercise the
+right to control the distribution of derivative or collective works based on
+the Program.
+
+In addition, mere aggregation of another work not based on the Program with the
+Program (or with a work based on the Program) on a volume of a storage or
+distribution medium does not bring the other work under the scope of this
+License.
+
+3. You may copy and distribute the Program (or a work based on it, under
+Section 2) in object code or executable form under the terms of Sections 1 and
+2 above provided that you also do one of the following:
+
+    a) Accompany it with the complete corresponding machine-readable source
+    code, which must be distributed under the terms of Sections 1 and 2 above
+    on a medium customarily used for software interchange; or,
+
+    b) Accompany it with a written offer, valid for at least three years, to
+    give any third party, for a charge no more than your cost of physically
+    performing source distribution, a complete machine-readable copy of the
+    corresponding source code, to be distributed under the terms of Sections 1
+    and 2 above on a medium customarily used for software interchange; or,
+
+    c) Accompany it with the information you received as to the offer to
+    distribute corresponding source code.  (This alternative is allowed only
+    for noncommercial distribution and only if you received the program in
+    object code or executable form with such an offer, in accord with
+    Subsection b above.)
+
+The source code for a work means the preferred form of the work for making
+modifications to it.  For an executable work, complete source code means all
+the source code for all modules it contains, plus any associated interface
+definition files, plus the scripts used to control compilation and installation
+of the executable.  However, as a special exception, the source code
+distributed need not include anything that is normally distributed (in either
+source or binary form) with the major components (compiler, kernel, and so on)
+of the operating system on which the executable runs, unless that component
+itself accompanies the executable.
+
+If distribution of executable or object code is made by offering access to copy
+from a designated place, then offering equivalent access to copy the source
+code from the same place counts as distribution of the source code, even though
+third parties are not compelled to copy the source along with the object code.
+
+4. You may not copy, modify, sublicense, or distribute the Program except as
+expressly provided under this License.  Any attempt otherwise to copy, modify,
+sublicense or distribute the Program is void, and will automatically terminate
+your rights under this License.  However, parties who have received copies, or
+rights, from you under this License will not have their licenses terminated so
+long as such parties remain in full compliance.
+
+5. You are not required to accept this License, since you have not signed it.
+However, nothing else grants you permission to modify or distribute the Program
+or its derivative works.  These actions are prohibited by law if you do not
+accept this License.  Therefore, by modifying or distributing the Program (or
+any work based on the Program), you indicate your acceptance of this License to
+do so, and all its terms and conditions for copying, distributing or modifying
+the Program or works based on it.
+
+6. Each time you redistribute the Program (or any work based on the Program),
+the recipient automatically receives a license from the original licensor to
+copy, distribute or modify the Program subject to these terms and conditions.
+You may not impose any further restrictions on the recipients' exercise of the
+rights granted herein.  You are not responsible for enforcing compliance by
+third parties to this License.
+
+7. If, as a consequence of a court judgment or allegation of patent
+infringement or for any other reason (not limited to patent issues), conditions
+are imposed on you (whether by court order, agreement or otherwise) that
+contradict the conditions of this License, they do not excuse you from the
+conditions of this License.  If you cannot distribute so as to satisfy
+simultaneously your obligations under this License and any other pertinent
+obligations, then as a consequence you may not distribute the Program at all.
+For example, if a patent license would not permit royalty-free redistribution
+of the Program by all those who receive copies directly or indirectly through
+you, then the only way you could satisfy both it and this License would be to
+refrain entirely from distribution of the Program.
+
+If any portion of this section is held invalid or unenforceable under any
+particular circumstance, the balance of the section is intended to apply and
+the section as a whole is intended to apply in other circumstances.
+
+It is not the purpose of this section to induce you to infringe any patents or
+other property right claims or to contest validity of any such claims; this
+section has the sole purpose of protecting the integrity of the free software
+distribution system, which is implemented by public license practices.  Many
+people have made generous contributions to the wide range of software
+distributed through that system in reliance on consistent application of that
+system; it is up to the author/donor to decide if he or she is willing to
+distribute software through any other system and a licensee cannot impose that
+choice.
+
+This section is intended to make thoroughly clear what is believed to be a
+consequence of the rest of this License.
+
+8. If the distribution and/or use of the Program is restricted in certain
+countries either by patents or by copyrighted interfaces, the original
+copyright holder who places the Program under this License may add an explicit
+geographical distribution limitation excluding those countries, so that
+distribution is permitted only in or among countries not thus excluded.  In
+such case, this License incorporates the limitation as if written in the body
+of this License.
+
+9. The Free Software Foundation may publish revised and/or new versions of the
+General Public License from time to time.  Such new versions will be similar in
+spirit to the present version, but may differ in detail to address new problems
+or concerns.
+
+Each version is given a distinguishing version number.  If the Program
+specifies a version number of this License which applies to it and "any later
+version", you have the option of following the terms and conditions either of
+that version or of any later version published by the Free Software Foundation.
+If the Program does not specify a version number of this License, you may
+choose any version ever published by the Free Software Foundation.
+
+10. If you wish to incorporate parts of the Program into other free programs
+whose distribution conditions are different, write to the author to ask for
+permission.  For software which is copyrighted by the Free Software Foundation,
+write to the Free Software Foundation; we sometimes make exceptions for this.
+Our decision will be guided by the two goals of preserving the free status of
+all derivatives of our free software and of promoting the sharing and reuse of
+software generally.
+
+NO WARRANTY
+
+11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY FOR
+THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW.  EXCEPT WHEN OTHERWISE
+STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES PROVIDE THE
+PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED,
+INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS FOR A PARTICULAR PURPOSE.  THE ENTIRE RISK AS TO THE QUALITY AND
+PERFORMANCE OF THE PROGRAM IS WITH YOU.  SHOULD THE PROGRAM PROVE DEFECTIVE,
+YOU ASSUME THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION.
+
+12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING WILL
+ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR REDISTRIBUTE THE
+PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY
+GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR
+INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA
+BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A
+FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER
+OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+END OF TERMS AND CONDITIONS
+
+How to Apply These Terms to Your New Programs
+
+If you develop a new program, and you want it to be of the greatest possible
+use to the public, the best way to achieve this is to make it free software
+which everyone can redistribute and change under these terms.
+
+To do so, attach the following notices to the program.  It is safest to attach
+them to the start of each source file to most effectively convey the exclusion
+of warranty; and each file should have at least the "copyright" line and a
+pointer to where the full notice is found.
+
+    One line to give the program's name and a brief idea of what it does.
+
+    Copyright (C) <year> <name of author>
+
+    This program is free software; you can redistribute it and/or modify it
+    under the terms of the GNU General Public License as published by the Free
+    Software Foundation; either version 2 of the License, or (at your option)
+    any later version.
+
+    This program is distributed in the hope that it will be useful, but WITHOUT
+    ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+    FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License for
+    more details.
+
+    You should have received a copy of the GNU General Public License along
+    with this program; if not, write to the Free Software Foundation, Inc., 59
+    Temple Place, Suite 330, Boston, MA 02111-1307 USA
+
+Also add information on how to contact you by electronic and paper mail.
+
+If the program is interactive, make it output a short notice like this when it
+starts in an interactive mode:
+
+    Gnomovision version 69, Copyright (C) year name of author Gnomovision comes
+    with ABSOLUTELY NO WARRANTY; for details type 'show w'.  This is free
+    software, and you are welcome to redistribute it under certain conditions;
+    type 'show c' for details.
+
+The hypothetical commands 'show w' and 'show c' should show the appropriate
+parts of the General Public License.  Of course, the commands you use may be
+called something other than 'show w' and 'show c'; they could even be
+mouse-clicks or menu items--whatever suits your program.
+
+You should also get your employer (if you work as a programmer) or your school,
+if any, to sign a "copyright disclaimer" for the program, if necessary.  Here
+is a sample; alter the names:
+
+    Yoyodyne, Inc., hereby disclaims all copyright interest in the program
+    'Gnomovision' (which makes passes at compilers) written by James Hacker.
+
+    signature of Ty Coon, 1 April 1989
+
+    Ty Coon, President of Vice
+
+This General Public License does not permit incorporating your program into
+proprietary programs.  If your program is a subroutine library, you may
+consider it more useful to permit linking proprietary applications with the
+library.  If this is what you want to do, use the GNU Library General Public
+License instead of this License.
+
+
+"CLASSPATH" EXCEPTION TO THE GPL
+
+Certain source files distributed by Oracle America and/or its affiliates are
+subject to the following clarification and special exception to the GPL, but
+only where Oracle has expressly included in the particular source file's header
+the words "Oracle designates this particular file as subject to the "Classpath"
+exception as provided by Oracle in the LICENSE file that accompanied this code."
+
+    Linking this library statically or dynamically with other modules is making
+    a combined work based on this library.  Thus, the terms and conditions of
+    the GNU General Public License cover the whole combination.
+
+    As a special exception, the copyright holders of this library give you
+    permission to link this library with independent modules to produce an
+    executable, regardless of the license terms of these independent modules,
+    and to copy and distribute the resulting executable under terms of your
+    choice, provided that you also meet, for each linked independent module,
+    the terms and conditions of the license of that module.  An independent
+    module is a module which is not derived from or based on this library.  If
+    you modify this library, you may extend this exception to your version of
+    the library, but you are not obligated to do so.  If you do not wish to do
+    so, delete this exception statement from your version.
+
+===========================================================================
+
+MIT License:
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+===========================================================================
\ No newline at end of file
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml b/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
index fbe9dbc..f557d7a 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
@@ -59,6 +59,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>algebricks-runtime</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
       <version>${project.version}</version>
     </dependency>
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
index c22d54d..3a2fe3b 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
@@ -19,6 +19,8 @@
 package org.apache.hyracks.algebricks.compiler.api;
 
 import java.util.List;
+import java.util.function.Function;
+import java.util.function.Supplier;
 
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -32,7 +34,9 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.IPartialAggregationTypeComputer;
 import org.apache.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.algebricks.data.IAWriterFactory;
 import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import org.apache.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
@@ -40,6 +44,7 @@
 import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
 import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import org.apache.hyracks.algebricks.data.IPrinterFactoryProvider;
+import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
 import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
 import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
 import org.apache.hyracks.algebricks.data.IUnnestingPositionWriterFactory;
@@ -49,8 +54,9 @@
 
 public abstract class AbstractCompilerFactoryBuilder {
 
-    protected List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites;
-    protected List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites;
+    protected Supplier<List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> logicalRewrites;
+    protected Function<IRuleSetKind, List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> logicalRewritesByKind;
+    protected Supplier<List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> physicalRewrites;
     protected ITypeTraitProvider typeTraitProvider;
     protected ISerializerDeserializerProvider serializerDeserializerProvider;
     protected IBinaryHashFunctionFactoryProvider hashFunctionFactoryProvider;
@@ -59,6 +65,8 @@
     protected IBinaryBooleanInspectorFactory binaryBooleanInspectorFactory;
     protected IBinaryIntegerInspectorFactory binaryIntegerInspectorFactory;
     protected IPrinterFactoryProvider printerProvider;
+    protected IAWriterFactory writerFactory;
+    protected IResultSerializerFactoryProvider resultSerializerFactoryProvider;
     protected IPredicateEvaluatorFactoryProvider predEvaluatorFactoryProvider;
     protected IExpressionRuntimeProvider expressionRuntimeProvider;
     protected IExpressionTypeComputer expressionTypeComputer;
@@ -78,11 +86,18 @@
 
     public abstract ICompilerFactory create();
 
-    public void setLogicalRewrites(List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites) {
+    public void setLogicalRewrites(
+            Supplier<List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> logicalRewrites) {
         this.logicalRewrites = logicalRewrites;
     }
 
-    public void setPhysicalRewrites(List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites) {
+    public void setLogicalRewritesByKind(
+            Function<IRuleSetKind, List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> logicalRewritesByKind) {
+        this.logicalRewritesByKind = logicalRewritesByKind;
+    }
+
+    public void setPhysicalRewrites(
+            Supplier<List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> physicalRewrites) {
         this.physicalRewrites = physicalRewrites;
     }
 
@@ -158,6 +173,22 @@
         return printerProvider;
     }
 
+    public void setWriterFactory(IAWriterFactory writerFactory) {
+        this.writerFactory = writerFactory;
+    }
+
+    public IAWriterFactory getWriterFactory() {
+        return writerFactory;
+    }
+
+    public void setResultSerializerFactoryProvider(IResultSerializerFactoryProvider resultSerializerFactoryProvider) {
+        this.resultSerializerFactoryProvider = resultSerializerFactoryProvider;
+    }
+
+    public IResultSerializerFactoryProvider getResultSerializerFactoryProvider() {
+        return resultSerializerFactoryProvider;
+    }
+
     public void setExpressionRuntimeProvider(IExpressionRuntimeProvider expressionRuntimeProvider) {
         this.expressionRuntimeProvider = expressionRuntimeProvider;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
index 891980f..e35a539 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
@@ -18,8 +18,11 @@
  */
 package org.apache.hyracks.algebricks.compiler.api;
 
+import java.util.List;
+
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IConflictingTypeResolver;
@@ -33,10 +36,15 @@
 import org.apache.hyracks.algebricks.core.config.AlgebricksConfig;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.algebricks.core.jobgen.impl.PlanCompiler;
+import org.apache.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
 import org.apache.hyracks.algebricks.core.rewriter.base.AlgebricksOptimizationContext;
 import org.apache.hyracks.algebricks.core.rewriter.base.HeuristicOptimizer;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.algebricks.data.IAWriterFactory;
+import org.apache.hyracks.algebricks.runtime.writers.SerializedDataWriterFactory;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 import org.apache.hyracks.api.job.IJobletEventListenerFactory;
 import org.apache.hyracks.api.job.JobSpecification;
@@ -58,14 +66,19 @@
                 IConflictingTypeResolver conflictingTypeResolver, PhysicalOptimizationConfig physicalOptimizationConfig,
                 AlgebricksPartitionConstraint clusterLocations, IWarningCollector warningCollector) {
             IPlanPrettyPrinter prettyPrinter = PlanPrettyPrinter.createStringPlanPrettyPrinter();
-            return new AlgebricksOptimizationContext(varCounter, expressionEvalSizeComputer,
+            return new AlgebricksOptimizationContext(this, varCounter, expressionEvalSizeComputer,
                     mergeAggregationExpressionFactory, expressionTypeComputer, missableTypeComputer,
                     conflictingTypeResolver, physicalOptimizationConfig, clusterLocations, prettyPrinter,
                     warningCollector);
         }
+
+        @Override
+        public IOptimizationContext cloneOptimizationContext(IOptimizationContext oc) {
+            return new AlgebricksOptimizationContext((AlgebricksOptimizationContext) oc);
+        }
     }
 
-    private IOptimizationContextFactory optCtxFactory;
+    private final IOptimizationContextFactory optCtxFactory;
 
     public HeuristicCompilerFactoryBuilder() {
         this.optCtxFactory = DefaultOptimizationContextFactory.INSTANCE;
@@ -77,42 +90,85 @@
 
     @Override
     public ICompilerFactory create() {
-        return new ICompilerFactory() {
-            @Override
-            public ICompiler createCompiler(final ILogicalPlan plan, final IMetadataProvider<?, ?> metadata,
-                    int varCounter) {
-                final IOptimizationContext oc = optCtxFactory.createOptimizationContext(varCounter,
-                        expressionEvalSizeComputer, mergeAggregationExpressionFactory, expressionTypeComputer,
-                        missableTypeComputer, conflictingTypeResolver, physicalOptimizationConfig, clusterLocations,
-                        warningCollector);
-                oc.setMetadataDeclarations(metadata);
-                final HeuristicOptimizer opt = new HeuristicOptimizer(plan, logicalRewrites, physicalRewrites, oc);
-                return new ICompiler() {
-
-                    @Override
-                    public void optimize() throws AlgebricksException {
-                        opt.optimize();
-                    }
-
-                    @Override
-                    public JobSpecification createJob(Object appContext,
-                            IJobletEventListenerFactory jobEventListenerFactory) throws AlgebricksException {
-                        AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Starting Job Generation.\n");
-                        JobGenContext context = new JobGenContext(null, metadata, appContext,
-                                serializerDeserializerProvider, hashFunctionFactoryProvider, hashFunctionFamilyProvider,
-                                comparatorFactoryProvider, typeTraitProvider, binaryBooleanInspectorFactory,
-                                binaryIntegerInspectorFactory, printerProvider, missingWriterFactory, nullWriterFactory,
-                                unnestingPositionWriterFactory, normalizedKeyComputerFactoryProvider,
-                                expressionRuntimeProvider, expressionTypeComputer, oc, expressionEvalSizeComputer,
-                                partialAggregationTypeComputer, predEvaluatorFactoryProvider,
-                                physicalOptimizationConfig.getFrameSize(), clusterLocations, warningCollector,
-                                maxWarnings, physicalOptimizationConfig);
-                        PlanCompiler pc = new PlanCompiler(context);
-                        return pc.compilePlan(plan, jobEventListenerFactory);
-                    }
-                };
-            }
-        };
+        return new CompilerFactoryImpl();
     }
 
+    private class CompilerFactoryImpl implements ICompilerFactory {
+        @Override
+        public ICompiler createCompiler(ILogicalPlan plan, IMetadataProvider<?, ?> metadata, int varCounter) {
+            IOptimizationContext optContext =
+                    optCtxFactory.createOptimizationContext(varCounter, expressionEvalSizeComputer,
+                            mergeAggregationExpressionFactory, expressionTypeComputer, missableTypeComputer,
+                            conflictingTypeResolver, physicalOptimizationConfig, clusterLocations, warningCollector);
+            optContext.setMetadataDeclarations(metadata);
+            optContext.setCompilerFactory(this);
+            return new CompilerImpl(this, plan, optContext, logicalRewrites.get(), physicalRewrites.get(),
+                    writerFactory);
+        }
+
+        @Override
+        public ICompiler createCompiler(ILogicalPlan plan, IOptimizationContext newOptContext,
+                IRuleSetKind ruleSetKind) {
+            if (newOptContext.getCompilerFactory() != this) {
+                throw new IllegalStateException();
+            }
+            return new CompilerImpl(this, plan, newOptContext, logicalRewritesByKind.apply(ruleSetKind),
+                    physicalRewrites.get(), SerializedDataWriterFactory.WITHOUT_RECORD_DESCRIPTOR);
+        }
+
+        private PlanCompiler createPlanCompiler(IOptimizationContext oc, Object appContext,
+                IAWriterFactory writerFactory) {
+            JobGenContext context = new JobGenContext(null, oc.getMetadataProvider(), appContext,
+                    serializerDeserializerProvider, hashFunctionFactoryProvider, hashFunctionFamilyProvider,
+                    comparatorFactoryProvider, typeTraitProvider, binaryBooleanInspectorFactory,
+                    binaryIntegerInspectorFactory, printerProvider, writerFactory, resultSerializerFactoryProvider,
+                    missingWriterFactory, nullWriterFactory, unnestingPositionWriterFactory,
+                    normalizedKeyComputerFactoryProvider, expressionRuntimeProvider, expressionTypeComputer, oc,
+                    expressionEvalSizeComputer, partialAggregationTypeComputer, predEvaluatorFactoryProvider,
+                    physicalOptimizationConfig.getFrameSize(), clusterLocations, warningCollector, maxWarnings,
+                    physicalOptimizationConfig);
+            return new PlanCompiler(context);
+        }
+    }
+
+    private static class CompilerImpl implements ICompiler {
+
+        private final CompilerFactoryImpl factory;
+
+        private final ILogicalPlan plan;
+
+        private final IOptimizationContext oc;
+
+        private final List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites;
+
+        private final List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites;
+
+        private final IAWriterFactory writerFactory;
+
+        private CompilerImpl(CompilerFactoryImpl factory, ILogicalPlan plan, IOptimizationContext oc,
+                List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites,
+                List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites,
+                IAWriterFactory writerFactory) {
+            this.factory = factory;
+            this.plan = plan;
+            this.oc = oc;
+            this.logicalRewrites = logicalRewrites;
+            this.physicalRewrites = physicalRewrites;
+            this.writerFactory = writerFactory;
+        }
+
+        @Override
+        public void optimize() throws AlgebricksException {
+            HeuristicOptimizer opt = new HeuristicOptimizer(plan, logicalRewrites, physicalRewrites, oc);
+            opt.optimize();
+        }
+
+        @Override
+        public JobSpecification createJob(Object appContext, IJobletEventListenerFactory jobEventListenerFactory)
+                throws AlgebricksException {
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Starting Job Generation.\n");
+            PlanCompiler pc = factory.createPlanCompiler(oc, appContext, writerFactory);
+            return pc.compilePlan(plan, jobEventListenerFactory);
+        }
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompilerFactory.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompilerFactory.java
index 7c138ea..07a8034 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompilerFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompilerFactory.java
@@ -19,8 +19,12 @@
 package org.apache.hyracks.algebricks.compiler.api;
 
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
 
 public interface ICompilerFactory {
     ICompiler createCompiler(ILogicalPlan plan, IMetadataProvider<?, ?> metadata, int varCounter);
+
+    ICompiler createCompiler(ILogicalPlan plan, IOptimizationContext newOptContext, IRuleSetKind ruleSetKind);
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/Counter.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/Counter.java
index 7f2d3c8..3982171 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/Counter.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/Counter.java
@@ -32,10 +32,18 @@
         return counter;
     }
 
+    public int getAndInc() {
+        return counter++;
+    }
+
     public void inc() {
         ++counter;
     }
 
+    public int incAndGet() {
+        return ++counter;
+    }
+
     public void set(int newStart) {
         counter = newStart;
     }
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 aac660a..1e4a388 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
@@ -41,32 +41,27 @@
     // constraints (e.g., FDs, equivalences)
 
     /**
-     * @param fds
-     *            Output argument: functional dependencies that can be inferred
-     *            from this expression.
-     * @param equivClasses
-     *            Output argument: Equivalence classes that can be inferred from
-     *            this expression.
+     * @param fds          Output argument: functional dependencies that can be inferred
+     *                     from this expression.
+     * @param equivClasses Output argument: Equivalence classes that can be inferred from
+     *                     this expression.
      */
     public void getConstraintsAndEquivClasses(Collection<FunctionalDependency> fds,
             Map<LogicalVariable, EquivalenceClass> equivClasses);
 
     /**
-     * @param fds
-     *            Output argument: functional dependencies that can be inferred
-     *            from this expression.
-     * @param outerVars
-     *            Input argument: variables coming from outer branch(es), e.g.,
-     *            the left branch of a left outer join.
+     * @param fds       Output argument: functional dependencies that can be inferred
+     *                  from this expression.
+     * @param outerVars Input argument: variables coming from outer branch(es), e.g.,
+     *                  the left branch of a left outer join.
      */
     public void getConstraintsForOuterJoin(Collection<FunctionalDependency> fds, Collection<LogicalVariable> outerVars);
 
     /**
-     * @param conjs
-     *            Output argument: a list of expression whose conjunction, in
-     *            any order, can replace the current expression.
+     * @param conjs Output argument: a list of expression whose conjunction, in
+     *              any order, can replace the current expression.
      * @return true if the expression can be broken in at least two conjuncts,
-     *         false otherwise.
+     * false otherwise.
      */
     public boolean splitIntoConjuncts(List<Mutable<ILogicalExpression>> conjs);
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
index 166ab9a..69ec210 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
@@ -33,6 +33,7 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 
@@ -93,4 +94,10 @@
     public PlanStructureVerifier getPlanStructureVerifier();
 
     public PlanStabilityVerifier getPlanStabilityVerifier();
+
+    void setCompilerFactory(Object factory);
+
+    Object getCompilerFactory();
+
+    IOptimizationContextFactory getOptimizationContextFactory();
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java
index cbe0882..3b44a97 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java
@@ -29,4 +29,10 @@
     // Integer
     public static final String MAX_NUMBER_FRAMES = "MAX_NUMBER_FRAMES"; // -->
     // Integer
+    public static final String OP_INPUT_CARDINALITY = "INPUT_CARDINALITY";
+    public static final String OP_OUTPUT_CARDINALITY = "OUTPUT_CARDINALITY";
+    public static final String OP_COST_TOTAL = "TOTAL_COST";
+    public static final String OP_COST_LOCAL = "OP_COST";
+    public static final String OP_LEFT_EXCHANGE_COST = "LEFT_EXCHANGE_COST";
+    public static final String OP_RIGHT_EXCHANGE_COST = "RIGHT_EXCHANGE_COST";
 }
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 e2a68da..7f7f65f 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
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.hyracks.algebricks.core.algebra.expressions;
 
 import java.util.Collection;
@@ -55,7 +56,7 @@
 
         @Override
         public String toString() {
-            return "TRUE";
+            return "true";
         }
 
         @Override
@@ -92,7 +93,7 @@
 
         @Override
         public String toString() {
-            return "FALSE";
+            return "false";
         }
 
         @Override
@@ -129,7 +130,7 @@
 
         @Override
         public String toString() {
-            return "NULL";
+            return "null";
         }
 
         @Override
@@ -166,7 +167,7 @@
 
         @Override
         public String toString() {
-            return "MISSING";
+            return "missing";
         }
 
         @Override
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/HashJoinExpressionAnnotation.java
similarity index 63%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/HashJoinExpressionAnnotation.java
index 6095b26..02ba2db 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/HashJoinExpressionAnnotation.java
@@ -17,8 +17,23 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+package org.apache.hyracks.algebricks.core.algebra.expressions;
 
-select value v from range(1,2) v where v > ?;
+import java.util.Objects;
+
+public class HashJoinExpressionAnnotation implements IExpressionAnnotation {
+    public enum BuildSide {
+        LEFT,
+        RIGHT
+    }
+
+    private final BuildSide side;
+
+    public HashJoinExpressionAnnotation(BuildSide side) {
+        this.side = Objects.requireNonNull(side);
+    }
+
+    public BuildSide getBuildSide() {
+        return side;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/JoinProductivityAnnotation.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/JoinProductivityAnnotation.java
new file mode 100644
index 0000000..0191b85
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/JoinProductivityAnnotation.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.algebricks.core.algebra.expressions;
+
+public class JoinProductivityAnnotation implements IExpressionAnnotation {
+
+    private final double productivity;
+    private final String leftSideDataSet;
+
+    public JoinProductivityAnnotation(double productivity, String leftSideDataSet) {
+        this.productivity = productivity;
+        this.leftSideDataSet = leftSideDataSet;
+    }
+
+    public double getJoinProductivity() {
+        return productivity;
+    }
+
+    public String getLeftSideDataSet() {
+        return leftSideDataSet;
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/PredicateCardinalityAnnotation.java
similarity index 68%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/PredicateCardinalityAnnotation.java
index 6095b26..e172449 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/PredicateCardinalityAnnotation.java
@@ -17,8 +17,17 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+package org.apache.hyracks.algebricks.core.algebra.expressions;
 
-select value v from range(1,2) v where v > ?;
+public class PredicateCardinalityAnnotation implements IExpressionAnnotation {
+
+    private final double selectivity;
+
+    public PredicateCardinalityAnnotation(double selectivity) {
+        this.selectivity = selectivity;
+    }
+
+    public double getSelectivity() {
+        return selectivity;
+    }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java
index 6e1ccef..34ed225 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java
@@ -38,6 +38,11 @@
         return true;
     }
 
+    default boolean isExternal() {
+        // A function is not external by default.
+        return false;
+    }
+
     /**
      * @param args,
      *            the arguments.
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 77fdd74..d350789 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
@@ -31,7 +31,9 @@
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.data.IAWriterFactory;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
 import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
@@ -56,11 +58,12 @@
             IProjectionInfo<?> projectionInfo) throws AlgebricksException;
 
     public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteFileRuntime(IDataSink sink,
-            int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc)
-            throws AlgebricksException;
+            int[] printColumns, IPrinterFactory[] printerFactories, IAWriterFactory writerFactory,
+            RecordDescriptor inputDesc) throws AlgebricksException;
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink,
-            int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc,
+            int[] printColumns, IPrinterFactory[] printerFactories, IAWriterFactory writerFactory,
+            IResultSerializerFactoryProvider resultSerializerFactoryProvider, RecordDescriptor inputDesc,
             IResultMetadata metadata, JobSpecification spec) throws AlgebricksException;
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getWriteResultRuntime(IDataSource<S> dataSource,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractDataSourceOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractDataSourceOperator.java
index 7d262a8..12a0572 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractDataSourceOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractDataSourceOperator.java
@@ -35,4 +35,8 @@
     public IDataSource<?> getDataSource() {
         return dataSource;
     }
+
+    public void setDataSource(IDataSource<?> datasource) {
+        this.dataSource = datasource;
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
index 5fc69b2..56e2e22 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
@@ -25,10 +25,10 @@
 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.VariablePropagationPolicy;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 
 public abstract class AbstractUnnestMapOperator extends AbstractUnnestOperator {
 
-    protected final Mutable<ILogicalExpression> expression;
     protected final List<Object> variableTypes;
     protected boolean propagateInput;
     protected List<Mutable<ILogicalExpression>> additionalFilteringExpressions;
@@ -42,7 +42,6 @@
     public AbstractUnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
             List<Object> variableTypes, boolean propagateInput) {
         super(variables, expression);
-        this.expression = expression;
         this.variableTypes = variableTypes;
         this.propagateInput = propagateInput;
         this.propagateIndexFilter = false;
@@ -90,6 +89,18 @@
         };
     }
 
+    @Override
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+        boolean changed = super.acceptExpressionTransform(visitor);
+
+        if (additionalFilteringExpressions != null) {
+            for (Mutable<ILogicalExpression> filterExpr : additionalFilteringExpressions) {
+                changed |= visitor.transform(filterExpr);
+            }
+        }
+        return changed;
+    }
+
     public boolean propagatesInput() {
         return propagateInput;
     }
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 bb18014..9f73113 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
@@ -57,7 +57,7 @@
     }
 
     public DataSourceScanOperator(List<LogicalVariable> variables, IDataSource<?> dataSource,
-            Mutable<ILogicalExpression> selectCondition, long outputLimit, IProjectionInfo projectionInfo) {
+            Mutable<ILogicalExpression> selectCondition, long outputLimit, IProjectionInfo<?> projectionInfo) {
         super(variables, dataSource);
         projectVars = new ArrayList<>();
         this.selectCondition = selectCondition;
@@ -77,7 +77,13 @@
 
     @Override
     public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
-        return false;
+        boolean changed = selectCondition != null && visitor.transform(selectCondition);
+        if (additionalFilteringExpressions != null) {
+            for (Mutable<ILogicalExpression> filterExpr : additionalFilteringExpressions) {
+                changed |= visitor.transform(filterExpr);
+            }
+        }
+        return changed;
     }
 
     @Override
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 c4bcc52..f8d07b8 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
@@ -28,6 +28,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
 import org.apache.hyracks.algebricks.core.algebra.typing.NonPropagatingTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 
 public class UnnestMapOperator extends AbstractUnnestMapOperator {
@@ -61,6 +62,12 @@
         return visitor.visitUnnestMapOperator(this, arg);
     }
 
+    @Override
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+        boolean changed = super.acceptExpressionTransform(visitor);
+        return selectCondition != null && visitor.transform(selectCondition) || changed;
+    }
+
     // When propagateInput is true,
     // this operator propagates all input variables.
     @Override
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 138cff8..1f7c16c 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
@@ -102,8 +102,9 @@
         IPrinterFactory[] pf =
                 JobGenHelper.mkPrinterFactories(inputSchemas[0], context.getTypeEnvironment(op), context, columns);
 
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = mp.getResultHandleRuntime(
-                resultOp.getDataSink(), columns, pf, inputDesc, resultOp.getResultMetadata(), spec);
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints =
+                mp.getResultHandleRuntime(resultOp.getDataSink(), columns, pf, context.getWriterFactory(),
+                        context.getResultSerializerFactoryProvider(), inputDesc, resultOp.getResultMetadata(), spec);
         IOperatorDescriptor opDesc = runtimeAndConstraints.first;
         opDesc.setSourceLocation(resultOp.getSourceLocation());
         builder.contributeHyracksOperator(resultOp, opDesc);
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 6c5c8d0..cae5137 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
@@ -75,16 +75,18 @@
         }
 
         IPartitioningProperty pp;
-
         AbstractLogicalOperator op = (AbstractLogicalOperator) iop;
 
+        // the partitioning property of the nested loop join is the same as the left branch.
+        // it cannot be the same as the right branch (BROADCAST) because the final joined data is not replicated at
+        // all partitions, and hence the final joined data is not BROADCAST.
         if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED) {
-            AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(1).getValue();
-            IPhysicalPropertiesVector pv1 = op2.getPhysicalOperator().getDeliveredProperties();
-            if (pv1 == null) {
+            AbstractLogicalOperator leftOp = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+            IPhysicalPropertiesVector leftOpProperties = leftOp.getPhysicalOperator().getDeliveredProperties();
+            if (leftOpProperties == null) {
                 pp = null;
             } else {
-                pp = pv1.getPartitioningProperty();
+                pp = leftOpProperties.getPartitioningProperty();
             }
         } else {
             pp = IPartitioningProperty.UNPARTITIONED;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
index 083e4d3..2bd78b1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
@@ -27,7 +27,6 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
-import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
 import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
@@ -43,7 +42,9 @@
     @Override
     public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
         AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
-        deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+        if (op2.getDeliveredPhysicalProperties() != null) {
+            deliveredProperties = op2.getDeliveredPhysicalProperties().clone();
+        }
     }
 
     @Override
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 3521a27..07c798f 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,7 +101,7 @@
         IMetadataProvider<?, ?> mp = context.getMetadataProvider();
 
         Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> runtimeAndConstraints =
-                mp.getWriteFileRuntime(write.getDataSink(), columns, pf, inputDesc);
+                mp.getWriteFileRuntime(write.getDataSink(), columns, pf, context.getWriterFactory(), inputDesc);
         IPushRuntimeFactory runtime = runtimeAndConstraints.first;
         runtime.setSourceLocation(write.getSourceLocation());
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/IPlanPrettyPrinter.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/IPlanPrettyPrinter.java
index faf4976..6afbabc 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/IPlanPrettyPrinter.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/IPlanPrettyPrinter.java
@@ -18,6 +18,8 @@
  */
 package org.apache.hyracks.algebricks.core.algebra.prettyprint;
 
+import java.util.Map;
+
 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.ILogicalPlan;
@@ -42,6 +44,9 @@
     /** Prints the whole logical plan. */
     IPlanPrettyPrinter printPlan(ILogicalPlan plan) throws AlgebricksException;
 
+    /** Prints the logical plan, annotated with physical operator and connector ids */
+    IPlanPrettyPrinter printPlan(ILogicalPlan plan, Map<Object, String> log2phys) throws AlgebricksException;
+
     /** Resets the state of the pretty printer. */
     IPlanPrettyPrinter reset() throws AlgebricksException;
 
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 64abaee..430ca1c 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
@@ -20,6 +20,7 @@
 
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -96,6 +97,14 @@
     }
 
     @Override
+    public final IPlanPrettyPrinter printPlan(ILogicalPlan plan, Map<Object, String> log2phys)
+            throws AlgebricksException {
+        //TODO(ian): would be nice if the text plan returned real operator ids too
+        printPlanImpl(plan, 0);
+        return this;
+    }
+
+    @Override
     public final IPlanPrettyPrinter printOperator(AbstractLogicalOperator op, boolean printInputs)
             throws AlgebricksException {
         printOperatorImpl(op, 0, printInputs);
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 2728e46..14bac72 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
@@ -19,6 +19,7 @@
 package org.apache.hyracks.algebricks.core.algebra.prettyprint;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Deque;
 import java.util.HashMap;
 import java.util.LinkedList;
@@ -33,7 +34,9 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
+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.OperatorAnnotations;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
@@ -96,8 +99,10 @@
     private static final String EXPRESSION_FIELD = "expression";
     private static final String CONDITION_FIELD = "condition";
     private static final String MISSING_VALUE_FIELD = "missing-value";
+    private static final String OP_CARDINALITY = "cardinality";
 
     private final Map<AbstractLogicalOperator, String> operatorIdentity = new HashMap<>();
+    private Map<Object, String> log2odid = Collections.emptyMap();
     private final IdCounter idCounter = new IdCounter();
     private final JsonGenerator jsonGenerator;
 
@@ -138,8 +143,7 @@
                 stringPrefix = stringPrefix.isEmpty() ? val.toString() : stringPrefix + "." + val.toString();
             }
             if (!operatorIdentity.containsKey(op)) {
-                String opId = stringPrefix.isEmpty() ? "" + Integer.toString(++id)
-                        : stringPrefix + "." + Integer.toString(++id);
+                String opId = stringPrefix.isEmpty() ? "" + (++id) : stringPrefix + "." + (++id);
                 operatorIdentity.put(op, opId);
             }
             return operatorIdentity.get(op);
@@ -162,6 +166,15 @@
     }
 
     @Override
+    public final IPlanPrettyPrinter printPlan(ILogicalPlan plan, Map<Object, String> log2phys)
+            throws AlgebricksException {
+        this.log2odid = log2phys;
+        printPlanImpl(plan);
+        flushContentToWriter();
+        return this;
+    }
+
+    @Override
     public final IPlanPrettyPrinter printOperator(AbstractLogicalOperator op, boolean printInputs)
             throws AlgebricksException {
         printOperatorImpl(op, printInputs);
@@ -191,11 +204,42 @@
             jsonGenerator.writeStartObject();
             op.accept(this, null);
             jsonGenerator.writeStringField("operatorId", idCounter.printOperatorId(op));
+            String od = log2odid.get(op);
+            if (od != null) {
+                jsonGenerator.writeStringField("runtime-id", od);
+            }
             IPhysicalOperator pOp = op.getPhysicalOperator();
             if (pOp != null) {
                 jsonGenerator.writeStringField("physical-operator", pOp.toString(false));
             }
             jsonGenerator.writeStringField("execution-mode", op.getExecutionMode().toString());
+
+            for (Map.Entry<String, Object> anno : op.getAnnotations().entrySet()) {
+                Object annotationVal = anno.getValue();
+                if (annotationVal != null) {
+                    String annotation = anno.getKey();
+                    switch (annotation) {
+                        case OperatorAnnotations.OP_COST_LOCAL:
+                        case OperatorAnnotations.OP_COST_TOTAL:
+                            jsonGenerator.writeStringField(annotation.toLowerCase().replace('_', '-'),
+                                    annotationVal.toString());
+                            break;
+                        case OperatorAnnotations.OP_INPUT_CARDINALITY:
+                            if (op.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+                                jsonGenerator.writeStringField(OP_CARDINALITY, annotationVal.toString());
+                            }
+                            break;
+                        case OperatorAnnotations.OP_OUTPUT_CARDINALITY:
+                            if (op.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN) {
+                                jsonGenerator.writeStringField(OP_CARDINALITY, annotationVal.toString());
+                            }
+                            break;
+                        default:
+                            break;
+                    }
+                }
+            }
+
             if (printInputs && !op.getInputs().isEmpty()) {
                 jsonGenerator.writeArrayFieldStart("inputs");
                 for (Mutable<ILogicalOperator> k : op.getInputs()) {
@@ -884,13 +928,17 @@
 
     /////////////// string fields ///////////////
 
-    /** Writes "fieldName": "expr" */
+    /**
+     * Writes "fieldName": "expr"
+     */
     private void writeStringFieldExpression(String fieldName, Mutable<ILogicalExpression> expressionRef, Void indent)
             throws AlgebricksException, IOException {
         writeStringFieldExpression(fieldName, expressionRef.getValue(), indent);
     }
 
-    /** Writes "fieldName": "expr" */
+    /**
+     * Writes "fieldName": "expr"
+     */
     private void writeStringFieldExpression(String fieldName, ILogicalExpression expression, Void indent)
             throws AlgebricksException, IOException {
         jsonGenerator.writeStringField(fieldName, expression.accept(exprVisitor, indent));
@@ -898,7 +946,9 @@
 
     /////////////// array fields ///////////////
 
-    /** Writes "fieldName": [ "var1", "var2", ... ] */
+    /**
+     * Writes "fieldName": [ "var1", "var2", ... ]
+     */
     private void writeArrayFieldOfVariables(String fieldName, List<LogicalVariable> variables) throws IOException {
         jsonGenerator.writeArrayFieldStart(fieldName);
         for (int i = 0, size = variables.size(); i < size; i++) {
@@ -907,7 +957,9 @@
         jsonGenerator.writeEndArray();
     }
 
-    /** Writes "fieldName": [ ["var1", "var2", ...], ["var1", "var2", ...] ] */
+    /**
+     * Writes "fieldName": [ ["var1", "var2", ...], ["var1", "var2", ...] ]
+     */
     private void writeArrayFieldOfNestedVariablesList(String fieldName, List<List<LogicalVariable>> nestedVarList)
             throws IOException {
         jsonGenerator.writeArrayFieldStart(fieldName);
@@ -920,7 +972,9 @@
         jsonGenerator.writeEndArray();
     }
 
-    /** Writes "fieldName" : [ "expr" ] */
+    /**
+     * Writes "fieldName" : [ "expr" ]
+     */
     private void writeArrayFieldOfExpression(String fieldName, Mutable<ILogicalExpression> expr, Void indent)
             throws IOException, AlgebricksException {
         jsonGenerator.writeArrayFieldStart(fieldName);
@@ -928,7 +982,9 @@
         jsonGenerator.writeEndArray();
     }
 
-    /** Writes "fieldName" : [ "expr1", "expr2", ...] */
+    /**
+     * Writes "fieldName" : [ "expr1", "expr2", ...]
+     */
     private void writeArrayFieldOfExpressions(String fieldName, List<Mutable<ILogicalExpression>> exprs, Void indent)
             throws IOException, AlgebricksException {
         jsonGenerator.writeArrayFieldStart(fieldName);
@@ -938,7 +994,9 @@
         jsonGenerator.writeEndArray();
     }
 
-    /** Writes "fieldName" : [ { "variable": "var1", "expression": "expr1" }, ... ] */
+    /**
+     * Writes "fieldName" : [ { "variable": "var1", "expression": "expr1" }, ... ]
+     */
     private void writeArrayFieldOfVariableExpressionPairs(String fieldName,
             List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> varExprPairs, Void indent)
             throws AlgebricksException, IOException {
@@ -954,7 +1012,9 @@
         jsonGenerator.writeEndArray();
     }
 
-    /** Writes "fieldName" : [ { "order": "", "expression": "" }, ... ] */
+    /**
+     * Writes "fieldName" : [ { "order": "", "expression": "" }, ... ]
+     */
     private void writeArrayFieldOfOrderExprList(String fieldName,
             List<Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>>> orderList, Void indent)
             throws AlgebricksException, IOException {
@@ -970,7 +1030,9 @@
 
     /////////////// object fields ///////////////
 
-    /** Writes "fieldName" : { "expressions": [ "expr1", "expr2", ...] } */
+    /**
+     * Writes "fieldName" : { "expressions": [ "expr1", "expr2", ...] }
+     */
     private void writeObjectFieldWithExpressions(String fieldName, List<Mutable<ILogicalExpression>> exprs, Void indent)
             throws IOException, AlgebricksException {
         jsonGenerator.writeObjectFieldStart(fieldName);
@@ -980,7 +1042,9 @@
 
     /////////////// other fields ///////////////
 
-    /** Writes "fieldName": null */
+    /**
+     * Writes "fieldName": null
+     */
     private void writeNullField(String fieldName) throws IOException {
         jsonGenerator.writeNullField(fieldName);
     }
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 26be09c..3e8a95f 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
@@ -37,8 +37,10 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 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.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
@@ -305,10 +307,8 @@
     /**
      * Compute type environment of a newly generated operator {@code op} and its input.
      *
-     * @param op,
-     *            the logical operator.
-     * @param context,the
-     *            optimization context.
+     * @param op,         the logical operator.
+     * @param context,the optimization context.
      * @throws AlgebricksException
      */
     public static void computeTypeEnvironmentBottomUp(ILogicalOperator op, ITypingContext context)
@@ -330,10 +330,8 @@
     /**
      * Computes the type environment for a logical query plan.
      *
-     * @param plan,
-     *            the logical plan to consider.
-     * @param context
-     *            the typing context.
+     * @param plan,   the logical plan to consider.
+     * @param context the typing context.
      * @throws AlgebricksException
      */
     public static void computeTypeEnvironment(ILogicalPlan plan, ITypingContext context) throws AlgebricksException {
@@ -402,8 +400,8 @@
     /**
      * Find an item a given list
      *
-     * @param list list to search in
-     * @param predicate predicate to test
+     * @param list           list to search in
+     * @param predicate      predicate to test
      * @param predicateParam parameter to pass to the predicate
      * @return item position in the given list or {@code -1} if not found
      */
@@ -445,9 +443,10 @@
     /**
      * Finds a variable assigned to a given expression and returns a new {@link VariableReferenceExpression}
      * referring to this variable.
-     * @param assignVarList list of variables
+     *
+     * @param assignVarList  list of variables
      * @param assignExprList list of expressions assigned to those variables
-     * @param searchExpr expression to search for
+     * @param searchExpr     expression to search for
      * @return said value, {@code null} if a variable is not found
      */
     public static VariableReferenceExpression findAssignedVariable(List<LogicalVariable> assignVarList,
@@ -509,4 +508,18 @@
             outVarRefList.add(new MutableObject<>(varRef));
         }
     }
+
+    public static void replaceVarWithExpr(AbstractFunctionCallExpression inExpr, LogicalVariable var,
+            ILogicalExpression replacementExpr) {
+        for (Mutable<ILogicalExpression> arg : inExpr.getArguments()) {
+            if (arg.getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                replaceVarWithExpr((AbstractFunctionCallExpression) (arg.getValue()), var, replacementExpr);
+            } else if (arg.getValue().getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                LogicalVariable v = ((VariableReferenceExpression) arg.getValue()).getVariableReference();
+                if (v.equals(var)) {
+                    arg.setValue(replacementExpr.cloneExpression());
+                }
+            }
+        }
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
index 86c6066..c94d72a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
@@ -35,6 +35,14 @@
     public static final boolean SUBPLAN_NESTEDPUSHDOWN_DEFAULT = true;
     public static final boolean MIN_MEMORY_ALLOCATION_DEFAULT = true;
     public static final boolean ARRAY_INDEX_DEFAULT = true;
+    public static final boolean CBO_DEFAULT = true;
+    public static final boolean CBO_TEST_DEFAULT = false;
+    public static final boolean FORCE_JOIN_ORDER_DEFAULT = false;
+    public static final String QUERY_PLAN_SHAPE_ZIGZAG = "zigzag";
+    public static final String QUERY_PLAN_SHAPE_LEFTDEEP = "leftdeep";
+    public static final String QUERY_PLAN_SHAPE_RIGHTDEEP = "rightdeep";
+    public static final String QUERY_PLAN_SHAPE_DEFAULT = QUERY_PLAN_SHAPE_ZIGZAG;
     public static final int EXTERNAL_SCAN_BUFFER_SIZE =
             StorageUtil.getIntSizeInBytes(8, StorageUtil.StorageUnit.KILOBYTE);
+    public static final boolean BATCH_LOOKUP_DEFAULT = false;
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobBuilder.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobBuilder.java
index c8fff99..d8bf190 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobBuilder.java
@@ -21,6 +21,7 @@
 import static org.apache.hyracks.api.exceptions.ErrorCode.DESCRIPTOR_GENERATION_ERROR;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -147,6 +148,15 @@
         hyracksOps.put(op, opDesc);
     }
 
+    public Map<Object, String> getLogical2PhysicalMap() {
+        Map<ILogicalOperator, String> mergedOperatorMap = new HashMap<>();
+        hyracksOps.forEach(((k, v) -> mergedOperatorMap.put(k, v.getOperatorId().toString())));
+        algebraicOpBelongingToMetaAsterixOp
+                .forEach((k, v) -> mergedOperatorMap.put(k, metaAsterixOps.get(v).getOperatorId().toString()));
+        connectors.forEach((k, v) -> mergedOperatorMap.put(k, v.getFirst().getConnectorId().toString()));
+        return Collections.unmodifiableMap(mergedOperatorMap);
+    }
+
     @Override
     public void contributeAlgebricksPartitionConstraint(IOperatorDescriptor opDesc,
             AlgebricksPartitionConstraint apcArg) {
@@ -174,6 +184,7 @@
             jobSpec.addRoot(opDesc);
         }
         setAllPartitionConstraints(tgtConstraints);
+        jobSpec.setLogical2PhysicalMap(getLogical2PhysicalMap());
     }
 
     public List<IOperatorDescriptor> getGeneratedMetaOps() {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
index 7c7d5a8..471380c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
@@ -35,6 +35,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.algebricks.data.IAWriterFactory;
 import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import org.apache.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
@@ -42,6 +43,7 @@
 import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
 import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import org.apache.hyracks.algebricks.data.IPrinterFactoryProvider;
+import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
 import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
 import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
 import org.apache.hyracks.algebricks.data.IUnnestingPositionWriterFactory;
@@ -57,6 +59,8 @@
     private final IBinaryHashFunctionFamilyProvider hashFunctionFamilyProvider;
     private final IBinaryComparatorFactoryProvider comparatorFactoryProvider;
     private final IPrinterFactoryProvider printerFactoryProvider;
+    private final IAWriterFactory writerFactory;
+    private final IResultSerializerFactoryProvider resultSerializerFactoryProvider;
     private final ITypeTraitProvider typeTraitProvider;
     private final IMetadataProvider<?, ?> metadataProvider;
     private final IMissingWriterFactory missingWriterFactory;
@@ -86,6 +90,7 @@
             IBinaryComparatorFactoryProvider comparatorFactoryProvider, ITypeTraitProvider typeTraitProvider,
             IBinaryBooleanInspectorFactory booleanInspectorFactory,
             IBinaryIntegerInspectorFactory integerInspectorFactory, IPrinterFactoryProvider printerFactoryProvider,
+            IAWriterFactory writerFactory, IResultSerializerFactoryProvider resultSerializerFactoryProvider,
             IMissingWriterFactory missingWriterFactory, IMissingWriterFactory nullWriterFactory,
             IUnnestingPositionWriterFactory unnestingPositionWriterFactory,
             INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider,
@@ -106,6 +111,8 @@
         this.booleanInspectorFactory = booleanInspectorFactory;
         this.integerInspectorFactory = integerInspectorFactory;
         this.printerFactoryProvider = printerFactoryProvider;
+        this.writerFactory = writerFactory;
+        this.resultSerializerFactoryProvider = resultSerializerFactoryProvider;
         this.clusterLocations = clusterLocations;
         this.normalizedKeyComputerFactoryProvider = normalizedKeyComputerFactoryProvider;
         this.missingWriterFactory = missingWriterFactory;
@@ -172,6 +179,14 @@
         return printerFactoryProvider;
     }
 
+    public IAWriterFactory getWriterFactory() {
+        return writerFactory;
+    }
+
+    public IResultSerializerFactoryProvider getResultSerializerFactoryProvider() {
+        return resultSerializerFactoryProvider;
+    }
+
     public IPredicateEvaluatorFactoryProvider getPredicateEvaluatorFactoryProvider() {
         return predEvaluatorFactoryProvider;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
index d1fd247..bee842c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
@@ -46,45 +46,29 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.ILogicalPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
 
 /**
  * The Algebricks default implementation for IOptimizationContext.
  */
-@SuppressWarnings({ "unchecked", "rawtypes" })
+@SuppressWarnings({ "rawtypes" })
 public class AlgebricksOptimizationContext implements IOptimizationContext {
 
-    private int varCounter;
+    private final IOptimizationContextFactory optContextFactory;
     private final IExpressionEvalSizeComputer expressionEvalSizeComputer;
     private final IMergeAggregationExpressionFactory mergeAggregationExpressionFactory;
     private final PhysicalOptimizationConfig physicalOptimizationConfig;
-    private final IVariableEvalSizeEnvironment varEvalSizeEnv = new IVariableEvalSizeEnvironment() {
 
-        Map<LogicalVariable, Integer> varSizeMap = new HashMap<>();
-
-        @Override
-        public void setVariableEvalSize(LogicalVariable var, int size) {
-            varSizeMap.put(var, size);
-        }
-
-        @Override
-        public int getVariableEvalSize(LogicalVariable var) {
-            return varSizeMap.get(var);
-        }
-    };
-
-    private Map<ILogicalOperator, IVariableTypeEnvironment> typeEnvMap = new HashMap<>();
-
-    private Map<ILogicalOperator, HashSet<ILogicalOperator>> alreadyCompared = new HashMap<>();
-    private Map<IAlgebraicRewriteRule, HashSet<ILogicalOperator>> dontApply = new HashMap<>();
-    private Map<LogicalVariable, FunctionalDependency> varToPrimaryKey = new HashMap<>();
-
-    private IMetadataProvider metadataProvider;
-    private HashSet<LogicalVariable> notToBeInlinedVars = new HashSet<>();
-
+    private final VariableEvalSizeEnvironmentImpl varEvalSizeEnv = new VariableEvalSizeEnvironmentImpl();
+    private final Map<ILogicalOperator, IVariableTypeEnvironment> typeEnvMap = new HashMap<>();
+    private final Map<ILogicalOperator, HashSet<ILogicalOperator>> alreadyCompared = new HashMap<>();
+    private final Map<IAlgebraicRewriteRule, HashSet<ILogicalOperator>> dontApply = new HashMap<>();
+    private final Map<LogicalVariable, FunctionalDependency> varToPrimaryKey = new HashMap<>();
+    private final HashSet<LogicalVariable> notToBeInlinedVars = new HashSet<>();
     protected final Map<ILogicalOperator, List<FunctionalDependency>> fdGlobalMap = new HashMap<>();
     protected final Map<ILogicalOperator, Map<LogicalVariable, EquivalenceClass>> eqClassGlobalMap = new HashMap<>();
-
     protected final Map<ILogicalOperator, ILogicalPropertiesVector> logicalProps = new HashMap<>();
+
     private final IExpressionTypeComputer expressionTypeComputer;
     private final IMissableTypeComputer nullableTypeComputer;
     private final INodeDomain defaultNodeDomain;
@@ -94,12 +78,18 @@
     private final PlanStructureVerifier planStructureVerifier;
     private final PlanStabilityVerifier planStabilityVerifier;
 
-    public AlgebricksOptimizationContext(int varCounter, IExpressionEvalSizeComputer expressionEvalSizeComputer,
+    private int varCounter;
+    private IMetadataProvider metadataProvider;
+    private Object compilerFactory;
+
+    public AlgebricksOptimizationContext(IOptimizationContextFactory optContextFactory, int varCounter,
+            IExpressionEvalSizeComputer expressionEvalSizeComputer,
             IMergeAggregationExpressionFactory mergeAggregationExpressionFactory,
             IExpressionTypeComputer expressionTypeComputer, IMissableTypeComputer nullableTypeComputer,
             IConflictingTypeResolver conflictingTypeResovler, PhysicalOptimizationConfig physicalOptimizationConfig,
             AlgebricksPartitionConstraint clusterLocations, IPlanPrettyPrinter prettyPrinter,
             IWarningCollector warningCollector) {
+        this.optContextFactory = optContextFactory;
         this.varCounter = varCounter;
         this.expressionEvalSizeComputer = expressionEvalSizeComputer;
         this.mergeAggregationExpressionFactory = mergeAggregationExpressionFactory;
@@ -115,6 +105,35 @@
         this.planStabilityVerifier = isSanityCheckEnabled ? new PlanStabilityVerifier(prettyPrinter) : null;
     }
 
+    public AlgebricksOptimizationContext(AlgebricksOptimizationContext from) {
+        optContextFactory = from.optContextFactory;
+        varCounter = from.varCounter;
+        expressionEvalSizeComputer = from.expressionEvalSizeComputer;
+        mergeAggregationExpressionFactory = from.mergeAggregationExpressionFactory;
+        expressionTypeComputer = from.expressionTypeComputer;
+        nullableTypeComputer = from.nullableTypeComputer;
+        physicalOptimizationConfig = from.physicalOptimizationConfig;
+        defaultNodeDomain = from.defaultNodeDomain;
+        prettyPrinter = from.prettyPrinter;
+        conflictingTypeResovler = from.conflictingTypeResovler;
+        warningCollector = NoOpWarningCollector.INSTANCE;
+        boolean isSanityCheckEnabled = physicalOptimizationConfig.isSanityCheckEnabled();
+        planStructureVerifier = isSanityCheckEnabled ? new PlanStructureVerifier(from.prettyPrinter, this) : null;
+        planStabilityVerifier = isSanityCheckEnabled ? new PlanStabilityVerifier(from.prettyPrinter) : null;
+        metadataProvider = from.metadataProvider;
+        compilerFactory = from.compilerFactory;
+
+        varEvalSizeEnv.varSizeMap.putAll(from.varEvalSizeEnv.varSizeMap);
+        typeEnvMap.putAll(from.typeEnvMap);
+        alreadyCompared.putAll(from.alreadyCompared);
+        dontApply.putAll(from.dontApply);
+        varToPrimaryKey.putAll(from.varToPrimaryKey);
+        notToBeInlinedVars.addAll(from.notToBeInlinedVars);
+        fdGlobalMap.putAll(from.fdGlobalMap);
+        eqClassGlobalMap.putAll(from.eqClassGlobalMap);
+        logicalProps.putAll(from.logicalProps);
+    }
+
     @Override
     public int getVarCounter() {
         return varCounter;
@@ -354,4 +373,34 @@
     public PlanStabilityVerifier getPlanStabilityVerifier() {
         return planStabilityVerifier;
     }
+
+    @Override
+    public IOptimizationContextFactory getOptimizationContextFactory() {
+        return optContextFactory;
+    }
+
+    @Override
+    public void setCompilerFactory(Object compilerFactory) {
+        this.compilerFactory = compilerFactory;
+    }
+
+    @Override
+    public Object getCompilerFactory() {
+        return compilerFactory;
+    }
+
+    protected static class VariableEvalSizeEnvironmentImpl implements IVariableEvalSizeEnvironment {
+
+        protected final Map<LogicalVariable, Integer> varSizeMap = new HashMap<>();
+
+        @Override
+        public void setVariableEvalSize(LogicalVariable var, int size) {
+            varSizeMap.put(var, size);
+        }
+
+        @Override
+        public int getVariableEvalSize(LogicalVariable var) {
+            return varSizeMap.get(var);
+        }
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
index f93d1f9..a717295 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
@@ -61,6 +61,7 @@
         logPlanAt("Plan Before Optimization", Level.TRACE);
         sanityCheckBeforeOptimization(plan);
         runLogicalOptimizationSets(plan, logicalRewrites);
+        logPlanAt("LogicalPlan ", Level.TRACE);
         computeSchemaBottomUpForPlan(plan);
         runPhysicalOptimizationSets(plan, physicalRewrites);
         logPlanAt("Plan After Optimization", Level.TRACE);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IOptimizationContextFactory.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IOptimizationContextFactory.java
index 1c41e9a..c81025d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IOptimizationContextFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IOptimizationContextFactory.java
@@ -28,10 +28,12 @@
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 
 public interface IOptimizationContextFactory {
-    public IOptimizationContext createOptimizationContext(int varCounter,
+    IOptimizationContext createOptimizationContext(int varCounter,
             IExpressionEvalSizeComputer expressionEvalSizeComputer,
             IMergeAggregationExpressionFactory mergeAggregationExpressionFactory,
             IExpressionTypeComputer expressionTypeComputer, IMissableTypeComputer missableTypeComputer,
             IConflictingTypeResolver conflictintTypeResolver, PhysicalOptimizationConfig physicalOptimizationConfig,
             AlgebricksPartitionConstraint clusterLocations, IWarningCollector warningCollector);
+
+    IOptimizationContext cloneOptimizationContext(IOptimizationContext oc);
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IRuleSetKind.java
similarity index 83%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IRuleSetKind.java
index 6095b26..5ce61ec 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IRuleSetKind.java
@@ -17,8 +17,8 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+package org.apache.hyracks.algebricks.core.rewriter.base;
 
-select value v from range(1,2) v where v > ?;
+public interface IRuleSetKind {
+    String name();
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
index 9c9b21d..86be6d0 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
@@ -47,6 +47,11 @@
     private static final String MIN_MEMORY_ALLOCATION = "MIN_MEMORY_ALLOCATION";
     private static final String ARRAY_INDEX = "ARRAY_INDEX";
     private static final String EXTERNAL_SCAN_BUFFER_SIZE = "EXTERNAL_SCAN_BUFFER_SIZE";
+    private static final String BATCH_LOOKUP = "BATCH_LOOKUP";
+    private static final String CBO = "CBO";
+    private static final String CBO_TEST = "CBO_TEST";
+    private static final String FORCE_JOIN_ORDER = "FORCE_JOIN_ORDER";
+    private static final String QUERY_PLAN_SHAPE = "QUERY_PLAN_SHAPE";
 
     private final Properties properties = new Properties();
 
@@ -239,6 +244,52 @@
         return getInt(EXTERNAL_SCAN_BUFFER_SIZE, AlgebricksConfig.EXTERNAL_SCAN_BUFFER_SIZE);
     }
 
+    public boolean getCBOMode() {
+        return getBoolean(CBO, AlgebricksConfig.CBO_DEFAULT);
+    }
+
+    public boolean getCBOTestMode() {
+        return getBoolean(CBO_TEST, AlgebricksConfig.CBO_TEST_DEFAULT);
+    }
+
+    public boolean getForceJoinOrderMode() {
+        return getBoolean(FORCE_JOIN_ORDER, AlgebricksConfig.FORCE_JOIN_ORDER_DEFAULT);
+    }
+
+    public String getQueryPlanShapeMode() {
+        String queryPlanShapeMode = getString(QUERY_PLAN_SHAPE, AlgebricksConfig.QUERY_PLAN_SHAPE_DEFAULT);
+        if (!(queryPlanShapeMode.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_ZIGZAG)
+                || queryPlanShapeMode.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_LEFTDEEP)
+                || queryPlanShapeMode.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_RIGHTDEEP))) {
+            return AlgebricksConfig.QUERY_PLAN_SHAPE_DEFAULT;
+        }
+        return queryPlanShapeMode;
+    }
+
+    public void setCBOMode(boolean cbo) {
+        setBoolean(CBO, cbo);
+    }
+
+    public void setCBOTestMode(boolean cboTest) {
+        setBoolean(CBO_TEST, cboTest);
+    }
+
+    public void setForceJoinOrderMode(boolean forceJoinOrder) {
+        setBoolean(FORCE_JOIN_ORDER, forceJoinOrder);
+    }
+
+    public void setQueryPlanShapeMode(String queryPlanShape) {
+        setString(QUERY_PLAN_SHAPE, queryPlanShape);
+    }
+
+    public boolean isBatchLookupEnabled() {
+        return getBoolean(BATCH_LOOKUP, AlgebricksConfig.BATCH_LOOKUP_DEFAULT);
+    }
+
+    public void setBatchLookup(boolean batchedLookup) {
+        setBoolean(BATCH_LOOKUP, batchedLookup);
+    }
+
     public void setExternalScanBufferSize(int bufferSize) {
         setInt(EXTERNAL_SCAN_BUFFER_SIZE, bufferSize);
     }
@@ -269,4 +320,13 @@
         String value = properties.getProperty(property);
         return value == null ? defaultValue : Boolean.parseBoolean(value);
     }
+
+    private void setString(String property, String value) {
+        properties.setProperty(property, value);
+    }
+
+    private String getString(String property, String defaultValue) {
+        String value = properties.getProperty(property);
+        return value == null ? defaultValue : value;
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IResultSerializerFactoryProvider.java b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IResultSerializerFactoryProvider.java
index 7525180..75fd791 100644
--- a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IResultSerializerFactoryProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IResultSerializerFactoryProvider.java
@@ -32,10 +32,8 @@
      *            - A printer factory array to print the tuple containing different fields.
      * @param writerFactory
      *            - A writer factory to write the serialized data to the print stream.
-     * @param inputRecordDesc
-     *            - The record descriptor describing the input frame to be serialized.
      * @return A new instance of result serialized appender.
      */
-    public IResultSerializerFactory getAqlResultSerializerFactoryProvider(int[] fields,
-            IPrinterFactory[] printerFactories, IAWriterFactory writerFactory);
+    public IResultSerializerFactory getResultSerializerFactoryProvider(int[] fields, IPrinterFactory[] printerFactories,
+            IAWriterFactory writerFactory);
 }
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 bd24644..fba095a 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
@@ -99,7 +99,7 @@
                 changed = false;
                 // applying the rewriting until fixpoint
                 topDownMaterialization(roots);
-                genCandidates(context);
+                genCandidates();
                 removeTrivialShare();
                 if (!equivalenceClasses.isEmpty()) {
                     changed = rewrite(context);
@@ -110,9 +110,7 @@
                 equivalenceClasses.clear();
                 childrenToParents.clear();
                 opToCandidateInputs.clear();
-                clusterMap.clear();
-                clusterWaitForMap.clear();
-                lastUsedClusterId = 0; // Resets lastUsedClusterId to 0.
+                resetPlanClusterState();
             } while (changed);
             roots.clear();
         }
@@ -151,7 +149,7 @@
 
     private boolean rewriteForOneEquivalentClass(List<Mutable<ILogicalOperator>> members, IOptimizationContext context)
             throws AlgebricksException {
-        List<Mutable<ILogicalOperator>> group = new ArrayList<Mutable<ILogicalOperator>>();
+        List<Mutable<ILogicalOperator>> group = new ArrayList<>();
         boolean rewritten = false;
         while (members.size() > 0) {
             group.clear();
@@ -176,14 +174,14 @@
             AbstractLogicalOperator aopCandidate = (AbstractLogicalOperator) candidate.getValue();
             List<Mutable<ILogicalOperator>> originalCandidateParents = childrenToParents.get(candidate);
 
-            rop.setExecutionMode(((AbstractLogicalOperator) candidate.getValue()).getExecutionMode());
+            rop.setExecutionMode(candidate.getValue().getExecutionMode());
             if (aopCandidate.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
                 rop.getInputs().add(candidate);
             } else {
                 AbstractLogicalOperator beforeExchange = new ExchangeOperator();
                 beforeExchange.setPhysicalOperator(new OneToOneExchangePOperator());
                 beforeExchange.setExecutionMode(rop.getExecutionMode());
-                Mutable<ILogicalOperator> beforeExchangeRef = new MutableObject<ILogicalOperator>(beforeExchange);
+                Mutable<ILogicalOperator> beforeExchangeRef = new MutableObject<>(beforeExchange);
                 beforeExchange.getInputs().add(candidate);
                 context.computeAndSetTypeEnvironmentForOperator(beforeExchange);
                 beforeExchange.recomputeSchema();
@@ -202,7 +200,7 @@
                     AbstractLogicalOperator exchange = new ExchangeOperator();
                     exchange.setPhysicalOperator(new OneToOneExchangePOperator());
                     exchange.setExecutionMode(rop.getExecutionMode());
-                    MutableObject<ILogicalOperator> exchangeRef = new MutableObject<ILogicalOperator>(exchange);
+                    MutableObject<ILogicalOperator> exchangeRef = new MutableObject<>(exchange);
                     exchange.getInputs().add(new MutableObject<>(rop));
                     rop.getOutputs().add(exchangeRef);
                     context.computeAndSetTypeEnvironmentForOperator(exchange);
@@ -246,10 +244,10 @@
                 exchOp.setPhysicalOperator(new OneToOneExchangePOperator());
                 exchOp.setExecutionMode(rop.getExecutionMode());
                 exchOp.getInputs().add(new MutableObject<>(rop));
-                MutableObject<ILogicalOperator> exchOpRef = new MutableObject<ILogicalOperator>(exchOp);
+                MutableObject<ILogicalOperator> exchOpRef = new MutableObject<>(exchOp);
                 rop.getOutputs().add(exchOpRef);
                 assignOperator.getInputs().add(exchOpRef);
-                projectOperator.getInputs().add(new MutableObject<ILogicalOperator>(assignOperator));
+                projectOperator.getInputs().add(new MutableObject<>(assignOperator));
 
                 // set the types
                 context.computeAndSetTypeEnvironmentForOperator(exchOp);
@@ -267,15 +265,15 @@
                             parentOp.getOperatorTag() == LogicalOperatorTag.PROJECT ? assignOperator : projectOperator;
                     if (parentOp.getPhysicalOperator().isMicroOperator()
                             || parentOp.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
-                        parentOp.getInputs().set(index, new MutableObject<ILogicalOperator>(childOp));
+                        parentOp.getInputs().set(index, new MutableObject<>(childOp));
                     } else {
                         // If the parent operator is a hyracks operator,
                         // an extra one-to-one exchange is needed.
                         AbstractLogicalOperator exchg = new ExchangeOperator();
                         exchg.setPhysicalOperator(new OneToOneExchangePOperator());
                         exchg.setExecutionMode(childOp.getExecutionMode());
-                        exchg.getInputs().add(new MutableObject<ILogicalOperator>(childOp));
-                        parentOp.getInputs().set(index, new MutableObject<ILogicalOperator>(exchg));
+                        exchg.getInputs().add(new MutableObject<>(childOp));
+                        parentOp.getInputs().set(index, new MutableObject<>(exchg));
                         context.computeAndSetTypeEnvironmentForOperator(exchg);
                         exchg.recomputeSchema();
                     }
@@ -285,10 +283,17 @@
             }
             cleanupPlan();
             rewritten = true;
+            resetPlanClusterState();
         }
         return rewritten;
     }
 
+    private void resetPlanClusterState() {
+        clusterMap.clear();
+        clusterWaitForMap.clear();
+        lastUsedClusterId = 0;
+    }
+
     /**
      * Cleans up the plan after combining similar branches into one branch making sure parents & children point to
      * each other correctly.
@@ -348,17 +353,15 @@
         }
     }
 
-    private void genCandidates(IOptimizationContext context) throws AlgebricksException {
-        List<List<Mutable<ILogicalOperator>>> previousEquivalenceClasses =
-                new ArrayList<List<Mutable<ILogicalOperator>>>();
+    private void genCandidates() throws AlgebricksException {
+        List<List<Mutable<ILogicalOperator>>> previousEquivalenceClasses = new ArrayList<>();
         while (equivalenceClasses.size() > 0) {
             previousEquivalenceClasses.clear();
             for (List<Mutable<ILogicalOperator>> candidates : equivalenceClasses) {
-                List<Mutable<ILogicalOperator>> candidatesCopy = new ArrayList<Mutable<ILogicalOperator>>();
-                candidatesCopy.addAll(candidates);
+                List<Mutable<ILogicalOperator>> candidatesCopy = new ArrayList<>(candidates);
                 previousEquivalenceClasses.add(candidatesCopy);
             }
-            List<Mutable<ILogicalOperator>> currentLevelOpRefs = new ArrayList<Mutable<ILogicalOperator>>();
+            List<Mutable<ILogicalOperator>> currentLevelOpRefs = new ArrayList<>();
             for (List<Mutable<ILogicalOperator>> candidates : equivalenceClasses) {
                 if (candidates.size() > 0) {
                     for (Mutable<ILogicalOperator> opRef : candidates) {
@@ -376,11 +379,11 @@
             if (currentLevelOpRefs.size() == 0) {
                 break;
             }
-            prune(context);
+            prune();
         }
         if (equivalenceClasses.size() < 1 && previousEquivalenceClasses.size() > 0) {
             equivalenceClasses.addAll(previousEquivalenceClasses);
-            prune(context);
+            prune();
         }
     }
 
@@ -412,8 +415,7 @@
     }
 
     private void candidatesGrow(List<Mutable<ILogicalOperator>> opList, List<Mutable<ILogicalOperator>> candidates) {
-        List<Mutable<ILogicalOperator>> previousCandidates = new ArrayList<Mutable<ILogicalOperator>>();
-        previousCandidates.addAll(candidates);
+        List<Mutable<ILogicalOperator>> previousCandidates = new ArrayList<>(candidates);
         candidates.clear();
         boolean validCandidate = false;
         for (Mutable<ILogicalOperator> op : opList) {
@@ -450,23 +452,18 @@
         }
     }
 
-    private void prune(IOptimizationContext context) throws AlgebricksException {
-        List<List<Mutable<ILogicalOperator>>> previousEquivalenceClasses =
-                new ArrayList<List<Mutable<ILogicalOperator>>>();
+    private void prune() throws AlgebricksException {
+        List<List<Mutable<ILogicalOperator>>> previousEquivalenceClasses = new ArrayList<>();
         for (List<Mutable<ILogicalOperator>> candidates : equivalenceClasses) {
-            List<Mutable<ILogicalOperator>> candidatesCopy = new ArrayList<Mutable<ILogicalOperator>>();
-            candidatesCopy.addAll(candidates);
+            List<Mutable<ILogicalOperator>> candidatesCopy = new ArrayList<>(candidates);
             previousEquivalenceClasses.add(candidatesCopy);
         }
         equivalenceClasses.clear();
         for (List<Mutable<ILogicalOperator>> candidates : previousEquivalenceClasses) {
             boolean[] reserved = new boolean[candidates.size()];
-            for (int i = 0; i < reserved.length; i++) {
-                reserved[i] = false;
-            }
             for (int i = candidates.size() - 1; i >= 0; i--) {
-                if (reserved[i] == false) {
-                    List<Mutable<ILogicalOperator>> equivalentClass = new ArrayList<Mutable<ILogicalOperator>>();
+                if (!reserved[i]) {
+                    List<Mutable<ILogicalOperator>> equivalentClass = new ArrayList<>();
                     ILogicalOperator candidate = candidates.get(i).getValue();
                     equivalentClass.add(candidates.get(i));
                     for (int j = i - 1; j >= 0; j--) {
@@ -502,7 +499,7 @@
         boolean worthMaterialization = worthMaterialization(group.get(0));
         boolean requiresMaterialization;
         // get clusterIds for each candidate in the group
-        List<Integer> groupClusterIds = new ArrayList<Integer>(group.size());
+        List<Integer> groupClusterIds = new ArrayList<>(group.size());
         for (int i = 0; i < group.size(); i++) {
             groupClusterIds.add(clusterMap.get(group.get(i)).getValue());
         }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
index e13ec30..129aba3 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
@@ -132,6 +132,7 @@
         e.setExecutionMode(inOp.getExecutionMode());
         context.computeAndSetTypeEnvironmentForOperator(e);
         e.recomputeSchema();
+        e.computeDeliveredPhysicalProperties(context);
 
         inOpRef.setValue(e);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java
index e1b2e5d..1eda133 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java
@@ -47,7 +47,7 @@
         }
         DistributeResultOperator dop = (DistributeResultOperator) op;
         IResultMetadata resultMetadata = dop.getResultMetadata();
-        if (resultMetadata.getOutputTypes() != null) {
+        if (resultMetadata == null || resultMetadata.getOutputTypes() != null) {
             return false;
         }
         List<Mutable<ILogicalExpression>> exprList = dop.getExpressions();
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java
index 3760be5..1ed7170 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java
@@ -138,9 +138,12 @@
                 if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
                     continue;
                 }
-                VariableReferenceExpression rhsVarRefExpr = (VariableReferenceExpression) expr;
-                // Update equivalence class map.
                 LogicalVariable lhs = assignOp.getVariables().get(i);
+                if (context.shouldNotBeInlined(lhs)) {
+                    continue;
+                }
+                // Update equivalence class map.
+                VariableReferenceExpression rhsVarRefExpr = (VariableReferenceExpression) expr;
                 LogicalVariable rhs = rhsVarRefExpr.getVariableReference();
                 updateEquivalenceClassMap(lhs, rhs);
             }
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 07d4e94..c9754f2 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
@@ -28,7 +28,7 @@
 import org.apache.hyracks.api.comm.VSizeFrame;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.EnforceFrameWriter;
-import org.apache.hyracks.api.dataflow.TimedFrameWriter;
+import org.apache.hyracks.api.dataflow.ProfiledFrameWriter;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.JobFlag;
@@ -67,7 +67,7 @@
                 new RunningAggregatorOutput(ctx, subplans, keyFieldIdx.length + decorFieldIdx.length, writer);
         IFrameWriter fw = outputWriter;
         if (profile) {
-            fw = TimedFrameWriter.time(outputWriter, ctx, "Aggregate Writer");
+            fw = ProfiledFrameWriter.time(outputWriter, ctx, "Aggregate Writer");
         } else if (enforce) {
             fw = EnforceFrameWriter.enforce(outputWriter);
         }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamLimitRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamLimitRuntimeFactory.java
index 793f095..d9ba859 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamLimitRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/StreamLimitRuntimeFactory.java
@@ -55,9 +55,12 @@
 
     @Override
     public String toString() {
-        String s = "stream-limit " + maxObjectsEvalFactory.toString();
+        String s = "stream-limit ";
+        if (maxObjectsEvalFactory != null) {
+            s += maxObjectsEvalFactory.toString();
+        }
         if (offsetEvalFactory != null) {
-            return s + ", " + offsetEvalFactory.toString();
+            return s + maxObjectsEvalFactory == null ? "" : ", " + offsetEvalFactory.toString();
         } else {
             return s;
         }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java
index 763e6ff..90fa824 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java
@@ -40,7 +40,7 @@
     }
 
     @Override
-    public IResultSerializerFactory getAqlResultSerializerFactoryProvider(final int[] fields,
+    public IResultSerializerFactory getResultSerializerFactoryProvider(final int[] fields,
             final IPrinterFactory[] printerFactories, final IAWriterFactory writerFactory) {
         return new IResultSerializerFactory() {
             private static final long serialVersionUID = 1L;
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/writers/SerializedDataWriterFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/writers/SerializedDataWriterFactory.java
index bc7634d..8ce4f2b 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/writers/SerializedDataWriterFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/writers/SerializedDataWriterFactory.java
@@ -32,7 +32,15 @@
 
 public class SerializedDataWriterFactory implements IAWriterFactory {
 
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 2L;
+
+    public static final SerializedDataWriterFactory WITHOUT_RECORD_DESCRIPTOR = new SerializedDataWriterFactory(false);
+
+    private final boolean writeRecordDescriptor;
+
+    public SerializedDataWriterFactory(boolean writeRecordDescriptor) {
+        this.writeRecordDescriptor = writeRecordDescriptor;
+    }
 
     @Override
     public IAWriter createWriter(final int[] fields, final PrintStream ps, IPrinterFactory[] printerFactories,
@@ -41,15 +49,17 @@
 
             @Override
             public void init() throws HyracksDataException {
-                // dump the SerializerDeserializers to disk
-                try {
-                    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-                    ObjectOutputStream oos = new ObjectOutputStream(baos);
-                    oos.writeObject(inputRecordDescriptor);
-                    baos.writeTo(ps);
-                    oos.close();
-                } catch (IOException e) {
-                    throw HyracksDataException.create(e);
+                if (writeRecordDescriptor) {
+                    // dump the SerializerDeserializers to disk
+                    try {
+                        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+                        ObjectOutputStream oos = new ObjectOutputStream(baos);
+                        oos.writeObject(inputRecordDescriptor);
+                        baos.writeTo(ps);
+                        oos.close();
+                    } catch (IOException e) {
+                        throw HyracksDataException.create(e);
+                    }
                 }
             }
 
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 0ee3658..7a320b1 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
@@ -346,12 +346,19 @@
     }
 
     public static class WaitForCompletionFunction extends Function {
-        private static final long serialVersionUID = 1L;
+        private static final long serialVersionUID = 2L;
 
         private final JobId jobId;
 
+        private final List<String> statOperatorNames;
+
         public WaitForCompletionFunction(JobId jobId) {
+            this(jobId, null);
+        }
+
+        public WaitForCompletionFunction(JobId jobId, List<String> statOperatorNames) {
             this.jobId = jobId;
+            this.statOperatorNames = statOperatorNames;
         }
 
         @Override
@@ -362,6 +369,10 @@
         public JobId getJobId() {
             return jobId;
         }
+
+        public List<String> getStatOperatorNames() {
+            return statOperatorNames;
+        }
     }
 
     public static class GetNodeControllersInfoFunction extends Function {
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 1118a68..be965eb 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
@@ -31,6 +31,7 @@
 import org.apache.hyracks.api.job.JobInfo;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 
 /**
  * Interface used by clients to communicate with the Hyracks Cluster Controller.
@@ -157,6 +158,19 @@
     void waitForCompletion(JobId jobId) throws Exception;
 
     /**
+     * Waits until the specified job has completed, either successfully or has
+     * encountered a permanent failure.
+     *
+     * @param jobId
+     *            JobId of the Job
+     * @param statsOperatorNames
+     *            names of the operators which stats should be returned
+     * @return stats for the specified operators
+     * @throws Exception
+     */
+    List<IOperatorStats> waitForCompletion(JobId jobId, List<String> statsOperatorNames) throws Exception;
+
+    /**
      * Deploy files to the cluster
      *
      * @param files
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 4cc47d2..be47014 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
@@ -30,6 +30,7 @@
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobInfo;
 import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 import org.apache.hyracks.api.topology.ClusterTopology;
 
 public interface IHyracksClientInterface {
@@ -53,6 +54,8 @@
 
     public void waitForCompletion(JobId jobId) throws Exception;
 
+    public List<IOperatorStats> waitForCompletion(JobId jobId, List<String> statOperatorNames) throws Exception;
+
     public Map<String, NodeControllerInfo> getNodeControllersInfo() throws Exception;
 
     public ClusterTopology getClusterTopology() throws Exception;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
index 4561af1..ad5624c 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
@@ -42,6 +42,8 @@
 
     TaskAttemptId getTaskAttemptId();
 
+    int getPartitionCount();
+
     ICounterContext getCounterContext();
 
     ExecutorService getExecutorService();
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IIntrospectingOperator.java
similarity index 79%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IIntrospectingOperator.java
index 6095b26..64897ef 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IIntrospectingOperator.java
@@ -1,3 +1,4 @@
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -16,9 +17,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.hyracks.api.dataflow;
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 
-select value v from range(1,2) v where v > ?;
+public interface IIntrospectingOperator {
+    void setOperatorStats(IOperatorStats stats);
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledFrameWriter.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledFrameWriter.java
new file mode 100644
index 0000000..5b9495a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledFrameWriter.java
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.comm.FrameConstants;
+import org.apache.hyracks.api.comm.FrameHelper;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.IStatsCollector;
+import org.apache.hyracks.api.job.profiling.OperatorStats;
+import org.apache.hyracks.api.job.profiling.counters.ICounter;
+import org.apache.hyracks.util.IntSerDeUtils;
+
+public class ProfiledFrameWriter implements IFrameWriter, IPassableTimer {
+
+    // The downstream data consumer of this writer.
+    private final IFrameWriter writer;
+    private long frameStart = 0;
+    final ICounter timeCounter;
+    final ICounter tupleCounter;
+    final IStatsCollector collector;
+    final IOperatorStats stats;
+    final IOperatorStats parentStats;
+    private int minSz = Integer.MAX_VALUE;
+    private int maxSz = -1;
+    private long avgSz;
+    final String name;
+
+    public ProfiledFrameWriter(IFrameWriter writer, IStatsCollector collector, String name, IOperatorStats stats,
+            IOperatorStats parentStats) {
+        this.writer = writer;
+        this.collector = collector;
+        this.name = name;
+        this.stats = stats;
+        this.parentStats = parentStats;
+        this.timeCounter = stats.getTimeCounter();
+        this.tupleCounter = parentStats != null ? parentStats.getTupleCounter() : null;
+    }
+
+    @Override
+    public final void open() throws HyracksDataException {
+        try {
+            startClock();
+            writer.open();
+        } finally {
+            stopClock();
+        }
+    }
+
+    @Override
+    public final void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        try {
+            int tupleCountOffset = FrameHelper.getTupleCountOffset(buffer.limit());
+            int tupleCount = IntSerDeUtils.getInt(buffer.array(), tupleCountOffset);
+            if (tupleCounter != null) {
+                long prevCount = tupleCounter.get();
+                for (int i = 0; i < tupleCount; i++) {
+                    int tupleLen = getTupleLength(i, tupleCountOffset, buffer);
+                    if (maxSz < tupleLen) {
+                        maxSz = tupleLen;
+                    }
+                    if (minSz > tupleLen) {
+                        minSz = tupleLen;
+                    }
+                    long prev = avgSz * prevCount;
+                    avgSz = (prev + tupleLen) / (prevCount + 1);
+                    prevCount++;
+                }
+                parentStats.getMaxTupleSz().set(maxSz);
+                parentStats.getMinTupleSz().set(minSz);
+                parentStats.getAverageTupleSz().set(avgSz);
+                tupleCounter.update(tupleCount);
+            }
+            startClock();
+            writer.nextFrame(buffer);
+        } finally {
+            stopClock();
+        }
+    }
+
+    @Override
+    public final void flush() throws HyracksDataException {
+        try {
+            startClock();
+            writer.flush();
+        } finally {
+            stopClock();
+        }
+    }
+
+    @Override
+    public final void fail() throws HyracksDataException {
+        writer.fail();
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        try {
+            startClock();
+            writer.close();
+        } finally {
+            stopClock();
+        }
+    }
+
+    private void stopClock() {
+        pause();
+        collector.giveClock(this);
+    }
+
+    private void startClock() {
+        if (frameStart > 0) {
+            return;
+        }
+        frameStart = collector.takeClock(this);
+    }
+
+    @Override
+    public void resume() {
+        if (frameStart > 0) {
+            return;
+        }
+        long nt = System.nanoTime();
+        frameStart = nt;
+    }
+
+    @Override
+    public void pause() {
+        if (frameStart > 1) {
+            long nt = System.nanoTime();
+            long delta = nt - frameStart;
+            timeCounter.update(delta);
+            frameStart = -1;
+        }
+    }
+
+    private int getTupleStartOffset(int tupleIndex, int tupleCountOffset, ByteBuffer buffer) {
+        return tupleIndex == 0 ? FrameConstants.TUPLE_START_OFFSET
+                : IntSerDeUtils.getInt(buffer.array(), tupleCountOffset - FrameConstants.SIZE_LEN * tupleIndex);
+    }
+
+    private int getTupleEndOffset(int tupleIndex, int tupleCountOffset, ByteBuffer buffer) {
+        return IntSerDeUtils.getInt(buffer.array(), tupleCountOffset - FrameConstants.SIZE_LEN * (tupleIndex + 1));
+    }
+
+    public int getTupleLength(int tupleIndex, int tupleCountOffset, ByteBuffer buffer) {
+        return getTupleEndOffset(tupleIndex, tupleCountOffset, buffer)
+                - getTupleStartOffset(tupleIndex, tupleCountOffset, buffer);
+    }
+
+    public static IFrameWriter time(IFrameWriter writer, IHyracksTaskContext ctx, String name)
+            throws HyracksDataException {
+        if (!(writer instanceof ProfiledFrameWriter)) {
+            IStatsCollector statsCollector = ctx.getStatsCollector();
+            IOperatorStats stats = new OperatorStats(name);
+            statsCollector.add(stats);
+            return new ProfiledFrameWriter(writer, ctx.getStatsCollector(), name, stats, null);
+
+        } else
+            return writer;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledOperatorNodePushable.java
new file mode 100644
index 0000000..f787a1c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledOperatorNodePushable.java
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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;
+
+import java.util.HashMap;
+
+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;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.IStatsCollector;
+import org.apache.hyracks.api.job.profiling.OperatorStats;
+import org.apache.hyracks.api.rewriter.runtime.SuperActivityOperatorNodePushable;
+
+public class ProfiledOperatorNodePushable extends ProfiledFrameWriter implements IOperatorNodePushable, IPassableTimer {
+
+    IOperatorNodePushable op;
+    ProfiledOperatorNodePushable parentOp;
+    ActivityId acId;
+    HashMap<Integer, IFrameWriter> inputs;
+    long frameStart;
+
+    ProfiledOperatorNodePushable(IOperatorNodePushable op, ActivityId acId, IStatsCollector collector,
+            IOperatorStats stats, ActivityId parent, ProfiledOperatorNodePushable parentOp)
+            throws HyracksDataException {
+        super(null, collector, acId.toString() + " - " + op.getDisplayName(), stats,
+                parentOp != null ? parentOp.getStats() : null);
+        this.parentOp = parentOp;
+        this.op = op;
+        this.acId = acId;
+        inputs = new HashMap<>();
+    }
+
+    @Override
+    public void initialize() throws HyracksDataException {
+        synchronized (collector) {
+            startClock();
+            op.initialize();
+            stopClock();
+        }
+    }
+
+    @Override
+    public void deinitialize() throws HyracksDataException {
+        synchronized (collector) {
+            startClock();
+            op.deinitialize();
+            stopClock();
+        }
+    }
+
+    @Override
+    public int getInputArity() {
+        return op.getInputArity();
+    }
+
+    @Override
+    public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc)
+            throws HyracksDataException {
+        op.setOutputFrameWriter(index, writer, recordDesc);
+    }
+
+    @Override
+    public IFrameWriter getInputFrameWriter(int index) {
+        IFrameWriter ifw = op.getInputFrameWriter(index);
+        if (!(op instanceof ProfiledFrameWriter) && ifw.equals(op)) {
+            return new ProfiledFrameWriter(op.getInputFrameWriter(index), collector,
+                    acId.toString() + "-" + op.getDisplayName(), stats, parentStats);
+        }
+        return op.getInputFrameWriter(index);
+    }
+
+    @Override
+    public String getDisplayName() {
+        return op.getDisplayName();
+    }
+
+    private void stopClock() {
+        pause();
+        collector.giveClock(this);
+    }
+
+    private void startClock() {
+        if (frameStart > 0) {
+            return;
+        }
+        frameStart = collector.takeClock(this);
+    }
+
+    @Override
+    public void resume() {
+        if (frameStart > 0) {
+            return;
+        }
+        long nt = System.nanoTime();
+        frameStart = nt;
+    }
+
+    @Override
+    public void pause() {
+        if (frameStart > 0) {
+            long nt = System.nanoTime();
+            long delta = nt - frameStart;
+            timeCounter.update(delta);
+            frameStart = -1;
+        }
+    }
+
+    public IOperatorStats getStats() {
+        return stats;
+    }
+
+    public IOperatorStats getParentStats() {
+        return parentStats;
+    }
+
+    public static IOperatorNodePushable time(IOperatorNodePushable op, IHyracksTaskContext ctx, ActivityId acId,
+            ProfiledOperatorNodePushable source) throws HyracksDataException {
+        String name = acId.toString() + " - " + op.getDisplayName();
+        IStatsCollector statsCollector = ctx.getStatsCollector();
+        IOperatorStats stats = new OperatorStats(name, acId.getOperatorDescriptorId());
+        statsCollector.add(stats);
+        if (op instanceof IIntrospectingOperator) {
+            ((IIntrospectingOperator) op).setOperatorStats(stats);
+        }
+        if (!(op instanceof ProfiledOperatorNodePushable) && !(op instanceof SuperActivityOperatorNodePushable)) {
+            return new ProfiledOperatorNodePushable(op, acId, ctx.getStatsCollector(), stats, acId, source);
+        }
+        return op;
+    }
+
+    public static void onlyAddStats(IOperatorNodePushable op, IHyracksTaskContext ctx, ActivityId acId)
+            throws HyracksDataException {
+        String name = acId.toString() + " - " + op.getDisplayName();
+        IStatsCollector statsCollector = ctx.getStatsCollector();
+        IOperatorStats stats = new OperatorStats(name, acId.getOperatorDescriptorId());
+        if (op instanceof IIntrospectingOperator) {
+            ((IIntrospectingOperator) op).setOperatorStats(stats);
+            statsCollector.add(stats);
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/TimedFrameWriter.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/TimedFrameWriter.java
deleted file mode 100644
index 83a4b34..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/TimedFrameWriter.java
+++ /dev/null
@@ -1,130 +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;
-
-import java.nio.ByteBuffer;
-
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.profiling.IStatsCollector;
-import org.apache.hyracks.api.job.profiling.counters.ICounter;
-
-public class TimedFrameWriter implements IFrameWriter, IPassableTimer {
-
-    // The downstream data consumer of this writer.
-    private final IFrameWriter writer;
-    private long frameStart = 0;
-    final ICounter counter;
-    final IStatsCollector collector;
-    final String name;
-
-    public TimedFrameWriter(IFrameWriter writer, IStatsCollector collector, String name, ICounter counter) {
-        this.writer = writer;
-        this.collector = collector;
-        this.name = name;
-        this.counter = counter;
-    }
-
-    protected TimedFrameWriter(IFrameWriter writer, IStatsCollector collector, String name) {
-        this(writer, collector, name, collector.getOrAddOperatorStats(name).getTimeCounter());
-    }
-
-    @Override
-    public final void open() throws HyracksDataException {
-        try {
-            startClock();
-            writer.open();
-        } finally {
-            stopClock();
-        }
-    }
-
-    @Override
-    public final void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        try {
-            startClock();
-            writer.nextFrame(buffer);
-        } finally {
-            stopClock();
-        }
-    }
-
-    @Override
-    public final void flush() throws HyracksDataException {
-        try {
-            startClock();
-            writer.flush();
-        } finally {
-            stopClock();
-        }
-    }
-
-    @Override
-    public final void fail() throws HyracksDataException {
-        writer.fail();
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        try {
-            startClock();
-            writer.close();
-        } finally {
-            stopClock();
-        }
-    }
-
-    private void stopClock() {
-        pause();
-        collector.giveClock(this);
-    }
-
-    private void startClock() {
-        if (frameStart > 0) {
-            return;
-        }
-        frameStart = collector.takeClock(this);
-    }
-
-    @Override
-    public void resume() {
-        if (frameStart > 0) {
-            return;
-        }
-        long nt = System.nanoTime();
-        frameStart = nt;
-    }
-
-    @Override
-    public void pause() {
-        if (frameStart > 1) {
-            long nt = System.nanoTime();
-            long delta = nt - frameStart;
-            counter.update(delta);
-            frameStart = -1;
-        }
-    }
-
-    public static IFrameWriter time(IFrameWriter writer, IHyracksTaskContext ctx, String name)
-            throws HyracksDataException {
-        return writer instanceof TimedFrameWriter ? writer
-                : new TimedFrameWriter(writer, ctx.getStatsCollector(), name);
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/TimedOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/TimedOperatorNodePushable.java
deleted file mode 100644
index 2d46bea..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/TimedOperatorNodePushable.java
+++ /dev/null
@@ -1,123 +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;
-
-import java.util.HashMap;
-
-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;
-import org.apache.hyracks.api.job.profiling.IStatsCollector;
-import org.apache.hyracks.api.rewriter.runtime.SuperActivityOperatorNodePushable;
-
-public class TimedOperatorNodePushable extends TimedFrameWriter implements IOperatorNodePushable, IPassableTimer {
-
-    IOperatorNodePushable op;
-    HashMap<Integer, IFrameWriter> inputs;
-    long frameStart;
-
-    TimedOperatorNodePushable(IOperatorNodePushable op, IStatsCollector collector) throws HyracksDataException {
-        super(null, collector, op.getDisplayName());
-        this.op = op;
-        inputs = new HashMap<>();
-    }
-
-    @Override
-    public void initialize() throws HyracksDataException {
-        synchronized (collector) {
-            startClock();
-            op.initialize();
-            stopClock();
-        }
-    }
-
-    @Override
-    public void deinitialize() throws HyracksDataException {
-        synchronized (collector) {
-            startClock();
-            op.deinitialize();
-            stopClock();
-        }
-    }
-
-    @Override
-    public int getInputArity() {
-        return op.getInputArity();
-    }
-
-    @Override
-    public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc)
-            throws HyracksDataException {
-        op.setOutputFrameWriter(index, writer, recordDesc);
-    }
-
-    @Override
-    public IFrameWriter getInputFrameWriter(int index) {
-        IFrameWriter ifw = op.getInputFrameWriter(index);
-        if (!(op instanceof TimedFrameWriter) && ifw.equals(op)) {
-            return new TimedFrameWriter(op.getInputFrameWriter(index), collector, op.getDisplayName(), counter);
-        }
-        return op.getInputFrameWriter(index);
-    }
-
-    @Override
-    public String getDisplayName() {
-        return op.getDisplayName();
-    }
-
-    private void stopClock() {
-        pause();
-        collector.giveClock(this);
-    }
-
-    private void startClock() {
-        if (frameStart > 0) {
-            return;
-        }
-        frameStart = collector.takeClock(this);
-    }
-
-    @Override
-    public void resume() {
-        if (frameStart > 0) {
-            return;
-        }
-        long nt = System.nanoTime();
-        frameStart = nt;
-    }
-
-    @Override
-    public void pause() {
-        if (frameStart > 0) {
-            long nt = System.nanoTime();
-            long delta = nt - frameStart;
-            counter.update(delta);
-            frameStart = -1;
-        }
-    }
-
-    public static IOperatorNodePushable time(IOperatorNodePushable op, IHyracksTaskContext ctx)
-            throws HyracksDataException {
-        if (!(op instanceof TimedOperatorNodePushable) && !(op instanceof SuperActivityOperatorNodePushable)) {
-            return new TimedOperatorNodePushable(op, ctx.getStatsCollector());
-        }
-        return op;
-    }
-}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/NoOpWarningCollector.java
similarity index 62%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/NoOpWarningCollector.java
index 6095b26..caf2464 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/NoOpWarningCollector.java
@@ -17,8 +17,27 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+package org.apache.hyracks.api.exceptions;
 
-select value v from range(1,2) v where v > ?;
+public final class NoOpWarningCollector implements IWarningCollector {
+
+    public static final IWarningCollector INSTANCE = new NoOpWarningCollector();
+
+    private NoOpWarningCollector() {
+    }
+
+    @Override
+    public void warn(Warning warning) {
+        // no-op
+    }
+
+    @Override
+    public boolean shouldWarn() {
+        return false;
+    }
+
+    @Override
+    public long getTotalWarningsCount() {
+        return 0;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java
index 58336a0..2c51d3d 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java
@@ -21,6 +21,7 @@
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -49,7 +50,7 @@
 import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class JobSpecification implements Serializable, IOperatorDescriptorRegistry, IConnectorDescriptorRegistry {
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 2L;
 
     private static final int DEFAULT_FRAME_SIZE = 32768;
 
@@ -67,6 +68,8 @@
 
     private final Map<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>> connectorOpMap;
 
+    private transient Map<Object, String> logical2PhysicalMap;
+
     private final Map<String, Serializable> properties;
 
     private final Set<Constraint> userConstraints;
@@ -108,6 +111,7 @@
         opOutputMap = new HashMap<>();
         connectorOpMap = new HashMap<>();
         properties = new HashMap<>();
+        logical2PhysicalMap = Collections.emptyMap();
         userConstraints = new HashSet<>();
         operatorIdCounter = 0;
         connectorIdCounter = 0;
@@ -326,6 +330,14 @@
         return metaOps;
     }
 
+    public void setLogical2PhysicalMap(Map<Object, String> logical2PhysicalMap) {
+        this.logical2PhysicalMap = logical2PhysicalMap;
+    }
+
+    public Map<Object, String> getLogical2PhysicalMap() {
+        return logical2PhysicalMap;
+    }
+
     private <K, V> void insertIntoIndexedMap(Map<K, List<V>> map, K key, int index, V value) {
         List<V> vList = map.computeIfAbsent(key, k -> new ArrayList<>());
         extend(vList, index);
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IOperatorStats.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IOperatorStats.java
index 181249a..ee68c3e 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IOperatorStats.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IOperatorStats.java
@@ -20,6 +20,7 @@
 
 import java.io.Serializable;
 
+import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
 import org.apache.hyracks.api.io.IWritable;
 import org.apache.hyracks.api.job.profiling.counters.ICounter;
 
@@ -43,7 +44,46 @@
     ICounter getTimeCounter();
 
     /**
-     * @return A counter used to track the number of pages pinned by an opeartor
+     * @return A counter used to track the number of pages pinned by an operator
      */
-    ICounter getDiskIoCounter();
+    ICounter getPageReads();
+
+    /**
+     * @return A counter used to track the number of pages read from disk by an operator
+     */
+
+    ICounter coldReadCounter();
+
+    /**
+     * @return A counter used to set the average tuple size outputted by an operator
+     */
+
+    ICounter getAverageTupleSz();
+
+    /**
+     * @return A counter used to set the max tuple size outputted by an operator
+     */
+
+    ICounter getMaxTupleSz();
+
+    /**
+     * @return A counter used to set the min tuple size outputted by an operator
+     */
+
+    ICounter getMinTupleSz();
+
+    /**
+     * @return A counter used to track the number of tuples read by operators that originate data,
+     *         like index searches or other scan types
+     */
+
+    ICounter getInputTupleCounter();
+
+    ICounter getLevel();
+
+    ICounter getBytesRead();
+
+    ICounter getBytesWritten();
+
+    OperatorDescriptorId getId();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IStatsCollector.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IStatsCollector.java
index 8930d34..d2ad20c 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IStatsCollector.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IStatsCollector.java
@@ -38,9 +38,9 @@
     /**
      * @param operatorName
      * @return {@link IOperatorStats} for the operator with name <code>operatorName</code>
-     * if it already exists, and adds it if it does not.
+     * if one exists or else null.
      */
-    IOperatorStats getOrAddOperatorStats(String operatorName);
+    IOperatorStats getOperatorStats(String operatorName);
 
     /**
      * Get every registered operator stats object
@@ -67,5 +67,4 @@
      * @param currHolder the timer that needs to be paused
      */
     void giveClock(IPassableTimer currHolder);
-
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/NoOpOperatorStats.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/NoOpOperatorStats.java
new file mode 100644
index 0000000..2d8a6df
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/NoOpOperatorStats.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.job.profiling;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
+import org.apache.hyracks.api.job.profiling.counters.ICounter;
+
+public class NoOpOperatorStats implements IOperatorStats {
+
+    private static final long serialVersionUID = 9055940822300360135L;
+
+    public static final NoOpOperatorStats INSTANCE = new NoOpOperatorStats();
+
+    private static final ICounter NOOP_COUNTER = new ICounter() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public long update(long delta) {
+            return 0;
+        }
+
+        @Override
+        public long set(long value) {
+            return 0;
+        }
+
+        @Override
+        public String getName() {
+            return null;
+        }
+
+        @Override
+        public long get() {
+            return 0;
+        }
+    };
+
+    private static final OperatorDescriptorId INVALID_ODID = new OperatorDescriptorId(-1);
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+
+    }
+
+    @Override
+    public String getName() {
+        return "";
+    }
+
+    @Override
+    public ICounter getTupleCounter() {
+        return NOOP_COUNTER;
+    }
+
+    @Override
+    public ICounter getTimeCounter() {
+        return NOOP_COUNTER;
+    }
+
+    @Override
+    public ICounter getPageReads() {
+        return NOOP_COUNTER;
+    }
+
+    @Override
+    public ICounter coldReadCounter() {
+        return NOOP_COUNTER;
+    }
+
+    @Override
+    public ICounter getAverageTupleSz() {
+        return NOOP_COUNTER;
+    }
+
+    @Override
+    public ICounter getMaxTupleSz() {
+        return NOOP_COUNTER;
+    }
+
+    @Override
+    public ICounter getMinTupleSz() {
+        return NOOP_COUNTER;
+    }
+
+    @Override
+    public ICounter getInputTupleCounter() {
+        return NOOP_COUNTER;
+    }
+
+    @Override
+    public ICounter getLevel() {
+        return NOOP_COUNTER;
+    }
+
+    @Override
+    public ICounter getBytesRead() {
+        return NOOP_COUNTER;
+    }
+
+    @Override
+    public ICounter getBytesWritten() {
+        return NOOP_COUNTER;
+    }
+
+    @Override
+    public OperatorDescriptorId getId() {
+        return INVALID_ODID;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/OperatorStats.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/OperatorStats.java
index 08c1adc..9aad10d 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/OperatorStats.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/OperatorStats.java
@@ -23,24 +23,51 @@
 import java.io.IOException;
 
 import org.apache.hyracks.api.com.job.profiling.counters.Counter;
+import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
 import org.apache.hyracks.api.job.profiling.counters.ICounter;
 
 public class OperatorStats implements IOperatorStats {
-    private static final long serialVersionUID = 6401830963367567167L;
+    private static final long serialVersionUID = 6401830963367567169L;
 
     public final String operatorName;
+    public OperatorDescriptorId id;
     public final ICounter tupleCounter;
     public final ICounter timeCounter;
-    public final ICounter diskIoCounter;
+    public final ICounter pageReads;
+    public final ICounter coldReadCounter;
+    public final ICounter avgTupleSz;
+    public final ICounter minTupleSz;
+    public final ICounter maxTupleSz;
+    public final ICounter inputTupleCounter;
+    public final ICounter level;
+    public final ICounter bytesRead;
+    public final ICounter bytesWritten;
 
-    public OperatorStats(String operatorName) {
+    //TODO: this is quickly becoming gross it should just be a map where the value is obliged to be a Counter
+
+    public OperatorStats(String operatorName, OperatorDescriptorId id) {
         if (operatorName == null || operatorName.isEmpty()) {
             throw new IllegalArgumentException("operatorName must not be null or empty");
         }
         this.operatorName = operatorName;
+        this.id = id;
         tupleCounter = new Counter("tupleCounter");
         timeCounter = new Counter("timeCounter");
-        diskIoCounter = new Counter("diskIoCounter");
+        pageReads = new Counter("diskIoCounter");
+        coldReadCounter = new Counter("coldReadCounter");
+        avgTupleSz = new Counter("avgTupleSz");
+        minTupleSz = new Counter("minTupleSz");
+        maxTupleSz = new Counter("maxTupleSz");
+        inputTupleCounter = new Counter("inputTupleCounter");
+        level = new Counter("level");
+        bytesRead = new Counter("bytesRead");
+        bytesWritten = new Counter("bytesWritten");
+        level.set(-1);
+
+    }
+
+    public OperatorStats(String operatorName) {
+        this(operatorName, new OperatorDescriptorId(-1));
     }
 
     public static IOperatorStats create(DataInput input) throws IOException {
@@ -66,28 +93,97 @@
     }
 
     @Override
-    public ICounter getDiskIoCounter() {
-        return diskIoCounter;
+    public ICounter getPageReads() {
+        return pageReads;
+    }
+
+    @Override
+    public ICounter coldReadCounter() {
+        return coldReadCounter;
+    }
+
+    @Override
+    public ICounter getAverageTupleSz() {
+        return avgTupleSz;
+    }
+
+    @Override
+    public ICounter getMaxTupleSz() {
+        return maxTupleSz;
+    }
+
+    @Override
+    public ICounter getMinTupleSz() {
+        return minTupleSz;
+    }
+
+    @Override
+    public ICounter getInputTupleCounter() {
+        return inputTupleCounter;
+    }
+
+    @Override
+    public ICounter getLevel() {
+        return level;
+    }
+
+    @Override
+    public ICounter getBytesRead() {
+        return bytesRead;
+    }
+
+    @Override
+    public ICounter getBytesWritten() {
+        return bytesWritten;
+    }
+
+    @Override
+    public OperatorDescriptorId getId() {
+        return id;
     }
 
     @Override
     public void writeFields(DataOutput output) throws IOException {
         output.writeUTF(operatorName);
+        id.writeFields(output);
         output.writeLong(tupleCounter.get());
         output.writeLong(timeCounter.get());
-        output.writeLong(diskIoCounter.get());
+        output.writeLong(pageReads.get());
+        output.writeLong(coldReadCounter.get());
+        output.writeLong(avgTupleSz.get());
+        output.writeLong(minTupleSz.get());
+        output.writeLong(maxTupleSz.get());
+        output.writeLong(inputTupleCounter.get());
+        output.writeLong(level.get());
+        output.writeLong(bytesRead.get());
+        output.writeLong(bytesWritten.get());
     }
 
     @Override
     public void readFields(DataInput input) throws IOException {
+        id = OperatorDescriptorId.create(input);
         tupleCounter.set(input.readLong());
         timeCounter.set(input.readLong());
-        diskIoCounter.set(input.readLong());
+        pageReads.set(input.readLong());
+        coldReadCounter.set(input.readLong());
+        avgTupleSz.set(input.readLong());
+        minTupleSz.set(input.readLong());
+        maxTupleSz.set(input.readLong());
+        inputTupleCounter.set(input.readLong());
+        level.set(input.readLong());
+        bytesRead.set(input.readLong());
+        bytesWritten.set(input.readLong());
     }
 
     @Override
     public String toString() {
         return "{ " + "\"operatorName\": \"" + operatorName + "\", " + "\"" + tupleCounter.getName() + "\": "
-                + tupleCounter.get() + ", \"" + timeCounter.getName() + "\": " + timeCounter.get() + " }";
+                + tupleCounter.get() + ", \"" + timeCounter.getName() + "\": " + timeCounter.get() + ", \""
+                + coldReadCounter.getName() + "\": " + coldReadCounter.get() + avgTupleSz.getName() + "\": "
+                + avgTupleSz.get() + ", \"" + minTupleSz.getName() + "\": " + minTupleSz.get() + ", \""
+                + minTupleSz.getName() + "\": " + timeCounter.get() + ", \"" + inputTupleCounter.getName() + "\": "
+                + bytesRead.get() + ", \"" + bytesRead.getName() + "\": " + bytesWritten.get() + ", \""
+                + bytesWritten.getName() + "\": " + inputTupleCounter.get() + ", \"" + level.getName() + "\": "
+                + level.get() + " }";
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
index e43d72a..efd4e07 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
@@ -43,7 +43,7 @@
 import org.apache.hyracks.api.dataflow.IActivity;
 import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
-import org.apache.hyracks.api.dataflow.TimedOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.ProfiledOperatorNodePushable;
 import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -109,10 +109,12 @@
         for (Entry<ActivityId, IActivity> entry : startActivities.entrySet()) {
             IOperatorNodePushable opPushable = null;
             if (profile) {
-                opPushable = TimedOperatorNodePushable
-                        .time(entry.getValue().createPushRuntime(ctx, recordDescProvider, partition, nPartitions), ctx);
+                IOperatorNodePushable wrapped =
+                        entry.getValue().createPushRuntime(ctx, recordDescProvider, partition, nPartitions);
+                opPushable = ProfiledOperatorNodePushable.time(wrapped, ctx, entry.getKey(), null);
             } else {
                 opPushable = entry.getValue().createPushRuntime(ctx, recordDescProvider, partition, nPartitions);
+                ProfiledOperatorNodePushable.onlyAddStats(opPushable, ctx, entry.getKey());
             }
             operatorNodePushablesBFSOrder.add(opPushable);
             operatorNodePushables.put(entry.getKey(), opPushable);
@@ -141,11 +143,18 @@
             IOperatorNodePushable destOp = operatorNodePushables.get(destId);
             if (destOp == null) {
                 if (profile) {
-                    destOp = TimedOperatorNodePushable.time(channel.getRight().getLeft().createPushRuntime(ctx,
-                            recordDescProvider, partition, nPartitions), ctx);
+                    IOperatorNodePushable wrapped = channel.getRight().getLeft().createPushRuntime(ctx,
+                            recordDescProvider, partition, nPartitions);
+                    if (sourceOp instanceof ProfiledOperatorNodePushable) {
+                        destOp = ProfiledOperatorNodePushable.time(wrapped, ctx, destId,
+                                (ProfiledOperatorNodePushable) sourceOp);
+                    } else {
+                        destOp = ProfiledOperatorNodePushable.time(wrapped, ctx, destId, null);
+                    }
                 } else {
                     destOp = channel.getRight().getLeft().createPushRuntime(ctx, recordDescProvider, partition,
                             nPartitions);
+                    ProfiledOperatorNodePushable.onlyAddStats(destOp, ctx, destId);
                 }
                 operatorNodePushablesBFSOrder.add(destOp);
                 operatorNodePushables.put(destId, destOp);
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksConstants.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksConstants.java
index c34a671..3c829e6 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksConstants.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksConstants.java
@@ -28,6 +28,8 @@
 
     public static final String INDEX_CURSOR_STATS = "INDEX_CURSOR_STATS";
 
+    public static final String TUPLE_PROJECTOR = "TUPLE_PROJECTOR";
+
     private HyracksConstants() {
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-client/pom.xml b/hyracks-fullstack/hyracks/hyracks-client/pom.xml
index 409da1f8..66b71fa 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-client/pom.xml
@@ -101,6 +101,11 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-util</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java
index b335a93..5dd5fff 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java
+++ b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hyracks.client.result;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 
@@ -35,7 +36,7 @@
 import org.apache.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
 
 //TODO(madhusudancs): Should this implementation be moved to org.apache.hyracks.client?
-public class ResultDirectory implements IResultDirectory {
+public class ResultDirectory implements IResultDirectory, Closeable {
     private final IPCSystem ipc;
     private final IResultDirectory remoteResultDirectory;
 
@@ -64,4 +65,9 @@
     public IResultMetadata getResultMetadata(JobId jobId, ResultSetId rsId) throws Exception {
         return remoteResultDirectory.getResultMetadata(jobId, rsId);
     }
+
+    @Override
+    public void close() throws IOException {
+        ipc.stop();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java
index 4d8767f..8a88045 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java
+++ b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java
@@ -18,6 +18,9 @@
  */
 package org.apache.hyracks.client.result;
 
+import java.io.Closeable;
+import java.io.IOException;
+
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.comm.NetworkAddress;
 import org.apache.hyracks.api.context.IHyracksCommonContext;
@@ -25,15 +28,15 @@
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.network.ISocketChannelFactory;
-import org.apache.hyracks.api.result.IResultDirectory;
 import org.apache.hyracks.api.result.IResultSet;
 import org.apache.hyracks.api.result.IResultSetReader;
 import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.client.net.ClientNetworkManager;
 import org.apache.hyracks.control.nc.resources.memory.FrameManager;
+import org.apache.hyracks.util.NetworkUtil;
 
-public class ResultSet implements IResultSet {
-    private final IResultDirectory resultDirectory;
+public class ResultSet implements IResultSet, Closeable {
+    private final ResultDirectory resultDirectory;
 
     private final ClientNetworkManager netManager;
 
@@ -51,6 +54,15 @@
     }
 
     @Override
+    public void close() throws IOException {
+        try {
+            netManager.stop();
+        } finally {
+            NetworkUtil.closeQuietly(resultDirectory);
+        }
+    }
+
+    @Override
     public IResultSetReader createReader(JobId jobId, ResultSetId resultSetId) throws HyracksDataException {
         IResultSetReader reader = null;
         try {
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 e6973dd..6da8fdf 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
@@ -147,8 +147,8 @@
             case WAIT_FOR_COMPLETION:
                 HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf =
                         (HyracksClientInterfaceFunctions.WaitForCompletionFunction) fn;
-                ccs.getWorkQueue()
-                        .schedule(new WaitForJobCompletionWork(ccs, wfcf.getJobId(), new IPCResponder<>(handle, mid)));
+                ccs.getWorkQueue().schedule(new WaitForJobCompletionWork(ccs, wfcf.getJobId(),
+                        wfcf.getStatOperatorNames(), new IPCResponder<>(handle, mid)));
                 break;
             case GET_NODE_CONTROLLERS_INFO:
                 ccs.getWorkQueue().schedule(
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java
index 0cc09b4..d865b4f 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java
@@ -111,10 +111,11 @@
         pmm = new PartitionMatchMaker();
         participatingNodeIds = new HashSet<>();
         cleanupPendingNodeIds = new HashSet<>();
-        profile = new JobProfile(jobId);
         connectorPolicyMap = new HashMap<>();
         operatorLocations = new HashMap<>();
         createTime = System.currentTimeMillis();
+        profile = new JobProfile(jobId);
+        profile.setCreateTime(createTime);
     }
 
     //Run a deployed job spec
@@ -200,6 +201,7 @@
 
     public void setStartTime(long startTime) {
         this.startTime = startTime;
+        this.profile.setStartTime(startTime);
     }
 
     public String getStartTimeZoneId() {
@@ -208,6 +210,7 @@
 
     public void setStartTimeZoneId(String startTimeZoneId) {
         this.startTimeZoneId = startTimeZoneId;
+        this.profile.setStartTimeZoneId(startTimeZoneId);
     }
 
     public long getEndTime() {
@@ -216,6 +219,7 @@
 
     public void setEndTime(long endTime) {
         this.endTime = endTime;
+        this.profile.setEndTime(endTime);
     }
 
     public void registerOperatorLocation(OperatorDescriptorId op, int partition, String location) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
index 911bedf..14d92fb 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
@@ -33,11 +33,14 @@
 public class WaitForJobCompletionWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final JobId jobId;
+    private final List<String> statOperatorNames;
     private final IResultCallback<Object> callback;
 
-    public WaitForJobCompletionWork(ClusterControllerService ccs, JobId jobId, IResultCallback<Object> callback) {
+    public WaitForJobCompletionWork(ClusterControllerService ccs, JobId jobId, List<String> statOperatorNames,
+            IResultCallback<Object> callback) {
         this.ccs = ccs;
         this.jobId = jobId;
+        this.statOperatorNames = statOperatorNames;
         this.callback = callback;
     }
 
@@ -53,7 +56,7 @@
                         Thread.currentThread()
                                 .setName(Thread.currentThread().getName() + " : WaitForCompletionForJobId: " + jobId);
                         jobRun.waitForCompletion();
-                        callback.setValue(null);
+                        callback.setValue(jobRun.getJobProfile().getAggregatedStats(statOperatorNames));
                     } catch (Exception e) {
                         callback.setException(e);
                     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
index 01cb9bf..bb40e2b 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
@@ -99,6 +99,7 @@
         PYTHON_USE_BUNDLED_MSGPACK(BOOLEAN, true),
         PYTHON_ARGS(STRING_ARRAY, (String[]) null),
         PYTHON_ENV(STRING_ARRAY, (String[]) null),
+        PYTHON_DS_PATH(STRING, (String) null),
         CREDENTIAL_FILE(
                 OptionTypes.STRING,
                 (Function<IApplicationConfig, String>) appConfig -> FileUtil
@@ -248,6 +249,8 @@
                     return "Whether or not to attempt to automatically set PYTHON_CMD to a usable interpreter";
                 case PYTHON_ENV:
                     return "List of environment variables to set when invoking the Python interpreter for Python UDFs. E.g. FOO=1";
+                case PYTHON_DS_PATH:
+                    return "Path to systemd socket for fenced Python UDFs. Requires JDK17+, *nix operating system, and ";
                 case CREDENTIAL_FILE:
                     return "Path to HTTP basic credentials";
                 default:
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/StatsCollector.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/StatsCollector.java
index 878d7c3..41beac0 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/StatsCollector.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/StatsCollector.java
@@ -30,10 +30,11 @@
 import org.apache.hyracks.api.dataflow.IPassableTimer;
 import org.apache.hyracks.api.job.profiling.IOperatorStats;
 import org.apache.hyracks.api.job.profiling.IStatsCollector;
+import org.apache.hyracks.api.job.profiling.NoOpOperatorStats;
 import org.apache.hyracks.api.job.profiling.OperatorStats;
 
 public class StatsCollector implements IStatsCollector {
-    private static final long serialVersionUID = 6858817639895434578L;
+    private static final long serialVersionUID = 6858817639895434572L;
 
     private final Map<String, IOperatorStats> operatorStatsMap = new LinkedHashMap<>();
     private transient Deque<IPassableTimer> clockHolder = new ArrayDeque<>();
@@ -47,8 +48,8 @@
     }
 
     @Override
-    public IOperatorStats getOrAddOperatorStats(String operatorName) {
-        return operatorStatsMap.computeIfAbsent(operatorName, OperatorStats::new);
+    public IOperatorStats getOperatorStats(String operatorName) {
+        return operatorStatsMap.getOrDefault(operatorName, NoOpOperatorStats.INSTANCE);
     }
 
     @Override
@@ -66,9 +67,9 @@
     public IOperatorStats getAggregatedStats() {
         IOperatorStats aggregatedStats = new OperatorStats("aggregated");
         for (IOperatorStats stats : operatorStatsMap.values()) {
-            aggregatedStats.getTupleCounter().update(stats.getTupleCounter().get());
+            aggregatedStats.getInputTupleCounter().update(stats.getInputTupleCounter().get());
             aggregatedStats.getTimeCounter().update(stats.getTimeCounter().get());
-            aggregatedStats.getDiskIoCounter().update(stats.getDiskIoCounter().get());
+            aggregatedStats.getPageReads().update(stats.getPageReads().get());
         }
         return aggregatedStats;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/JobProfile.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/JobProfile.java
index c4eff85..1110597 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/JobProfile.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/JobProfile.java
@@ -21,20 +21,32 @@
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.hyracks.api.dataflow.TaskAttemptId;
+import org.apache.hyracks.api.dataflow.TaskId;
 import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.IStatsCollector;
+import org.apache.hyracks.api.job.profiling.NoOpOperatorStats;
+import org.apache.hyracks.api.job.profiling.OperatorStats;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ArrayNode;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class JobProfile extends AbstractProfile {
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 2L;
 
     private JobId jobId;
+    private long createTime;
+    private long startTime;
+    private String startTimeZoneId;
+    private long endTime;
 
     private Map<String, JobletProfile> jobletProfiles;
 
@@ -61,12 +73,32 @@
         return jobletProfiles;
     }
 
+    public void setCreateTime(long createTime) {
+        this.createTime = createTime;
+    }
+
+    public void setStartTime(long startTime) {
+        this.startTime = startTime;
+    }
+
+    public void setStartTimeZoneId(String startTimeZoneId) {
+        this.startTimeZoneId = startTimeZoneId;
+    }
+
+    public void setEndTime(long endTime) {
+        this.endTime = endTime;
+    }
+
     @Override
     public ObjectNode toJSON() {
         ObjectMapper om = new ObjectMapper();
         ObjectNode json = om.createObjectNode();
 
         json.put("job-id", jobId.toString());
+        json.put("create-time", createTime);
+        json.put("start-time", startTime);
+        json.put("queued-time", startTime - createTime);
+        json.put("end-time", endTime);
         populateCounters(json);
         ArrayNode jobletsArray = om.createArrayNode();
         for (JobletProfile p : jobletProfiles.values()) {
@@ -91,6 +123,10 @@
     @Override
     public void readFields(DataInput input) throws IOException {
         jobId = JobId.create(input);
+        createTime = input.readLong();
+        startTime = input.readLong();
+        endTime = input.readLong();
+        startTimeZoneId = input.readUTF();
         int size = input.readInt();
         jobletProfiles = new HashMap<>();
         for (int i = 0; i < size; i++) {
@@ -103,10 +139,55 @@
     @Override
     public void writeFields(DataOutput output) throws IOException {
         jobId.writeFields(output);
+        output.writeLong(createTime);
+        output.writeLong(startTime);
+        output.writeLong(endTime);
+        output.writeUTF(startTimeZoneId);
         output.writeInt(jobletProfiles.size());
         for (Entry<String, JobletProfile> entry : jobletProfiles.entrySet()) {
             output.writeUTF(entry.getKey());
             entry.getValue().writeFields(output);
         }
     }
+
+    public List<IOperatorStats> getAggregatedStats(List<String> operatorNames) {
+        if (jobletProfiles == null || operatorNames == null || operatorNames.isEmpty()) {
+            return null;
+        }
+        // gather final task attempts for each task
+        Map<TaskId, TaskProfile> taskProfileMap = new HashMap<>();
+        for (JobletProfile jobletProfile : jobletProfiles.values()) {
+            for (TaskProfile taskProfile : jobletProfile.getTaskProfiles().values()) {
+                TaskAttemptId taskAttemptId = taskProfile.getTaskId();
+                TaskId taskId = taskAttemptId.getTaskId();
+                TaskProfile existingProfile = taskProfileMap.get(taskId);
+                if (existingProfile == null || taskAttemptId.getAttempt() > existingProfile.getTaskId().getAttempt()) {
+                    taskProfileMap.put(taskId, taskProfile);
+                }
+            }
+        }
+        // compute aggregated counts
+        int n = operatorNames.size();
+        IOperatorStats[] outStats = new IOperatorStats[n];
+        for (TaskProfile taskProfile : taskProfileMap.values()) {
+            IStatsCollector statsCollector = taskProfile.getStatsCollector();
+            for (int i = 0; i < n; i++) {
+                String operatorName = operatorNames.get(i);
+                IOperatorStats opTaskStats = statsCollector.getOperatorStats(operatorName);
+                if (opTaskStats.equals(NoOpOperatorStats.INSTANCE)) {
+                    continue;
+                }
+                IOperatorStats opOutStats = outStats[i];
+                if (opOutStats == null) {
+                    opOutStats = new OperatorStats(operatorName);
+                    outStats[i] = opOutStats;
+                }
+                opOutStats.getTupleCounter().update(opTaskStats.getTupleCounter().get());
+                opOutStats.getTimeCounter().update(opTaskStats.getTimeCounter().get());
+                opOutStats.getPageReads().update(opTaskStats.getPageReads().get());
+            }
+        }
+        return Arrays.asList(outStats);
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/TaskProfile.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/TaskProfile.java
index dab6d26..4036f00 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/TaskProfile.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/TaskProfile.java
@@ -130,7 +130,28 @@
             jpe.put("name", key);
             jpe.put("time", Double
                     .parseDouble(new DecimalFormat("#.####").format((double) value.getTimeCounter().get() / 1000000)));
-            jpe.put("disk-io", value.getDiskIoCounter().get());
+            if (value.getId().getId() >= 0) {
+                jpe.put("runtime-id", value.getId().toString());
+            }
+            if (value.getPageReads().get() > 0) {
+                jpe.put("pages-read", value.getPageReads().get());
+                jpe.put("pages-read-cold", value.coldReadCounter().get());
+            }
+            if (value.getTupleCounter().get() > 0) {
+                jpe.put("cardinality-out", value.getTupleCounter().get());
+                jpe.put("avg-tuple-size", value.getAverageTupleSz().get());
+                jpe.put("min-tuple-size", value.getMinTupleSz().get());
+                jpe.put("max-tuple-size", value.getMaxTupleSz().get());
+            }
+            if (value.getLevel().get() > -1) {
+                jpe.put("level", value.getLevel().get());
+            }
+            if (value.getBytesRead().get() > 0) {
+                jpe.put("bytes-read", value.getBytesRead().get());
+            }
+            if (value.getBytesWritten().get() > 0) {
+                jpe.put("bytes-written", value.getBytesWritten().get());
+            }
             countersObj.add(jpe);
         });
         json.set("counters", countersObj);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
index 158e24e..900ac7e 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
@@ -87,6 +87,8 @@
 
     private final TaskAttemptId taskAttemptId;
 
+    private final int partitionCount;
+
     private final String displayName;
 
     private final ExecutorService executorService;
@@ -131,12 +133,13 @@
 
     private final Map<Long, IThreadStats> perThreadStats = new HashMap<>();
 
-    public Task(Joblet joblet, Set<JobFlag> jobFlags, TaskAttemptId taskId, String displayName,
+    public Task(Joblet joblet, Set<JobFlag> jobFlags, TaskAttemptId taskId, int partitionCount, String displayName,
             ExecutorService executor, NodeControllerService ncs,
             List<List<PartitionChannel>> inputChannelsFromConnectors) {
         this.joblet = joblet;
         this.jobFlags = jobFlags;
         this.taskAttemptId = taskId;
+        this.partitionCount = partitionCount;
         this.displayName = displayName;
         this.executorService = executor;
         fileFactory = new WorkspaceFileFactory(this, joblet.getIoManager());
@@ -225,6 +228,11 @@
     }
 
     @Override
+    public int getPartitionCount() {
+        return partitionCount;
+    }
+
+    @Override
     public ICounter getCounter(String name, boolean create) {
         Counter counter = counterMap.get(name);
         if (counter == null && create) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
index 7d84fe9..f6c144d 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
@@ -152,11 +152,12 @@
                 ActivityCluster ac = acg.getActivityMap().get(aid);
                 IActivity han = ac.getActivityMap().get(aid);
                 LOGGER.trace("Initializing {} -> {} for {}", taId, han, jobId);
+                int partitionCount = td.getPartitionCount();
                 final int partition = tid.getPartition();
                 List<IConnectorDescriptor> inputs = ac.getActivityInputMap().get(aid);
-                task = new Task(joblet, flags, taId, han.getDisplayName(), ncs.getExecutor(), ncs,
+                task = new Task(joblet, flags, taId, partitionCount, han.getDisplayName(), ncs.getExecutor(), ncs,
                         createInputChannels(td, inputs));
-                IOperatorNodePushable operator = han.createPushRuntime(task, rdp, partition, td.getPartitionCount());
+                IOperatorNodePushable operator = han.createPushRuntime(task, rdp, partition, partitionCount);
                 List<IPartitionCollector> collectors = new ArrayList<>();
                 if (inputs != null) {
                     for (int i = 0; i < inputs.size(); ++i) {
@@ -182,7 +183,7 @@
                                 createPartitionWriterFactory(task, cPolicy, jobId, conn, partition, taId, flags);
                         LOGGER.trace("input: {}: {}", i, conn.getConnectorId());
                         IFrameWriter writer = conn.createPartitioner(task, recordDesc, pwFactory, partition,
-                                td.getPartitionCount(), td.getOutputPartitionCounts()[i]);
+                                partitionCount, td.getOutputPartitionCounts()[i]);
                         writer = (enforce && !profile) ? EnforceFrameWriter.enforce(writer) : writer;
                         operator.setOutputFrameWriter(i, writer, recordDesc);
                     }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
index c370b58..26b9f10 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
@@ -69,7 +69,7 @@
 
     @Override
     public void close() throws HyracksDataException {
-        if (!failed) {
+        if (!failed && handle != null) {
             ioManager.close(handle);
         }
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorNodePushable.java
index 3fa88bb..5559663 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractOperatorNodePushable.java
@@ -18,11 +18,18 @@
  */
 package org.apache.hyracks.dataflow.std.base;
 
+import org.apache.hyracks.api.dataflow.IIntrospectingOperator;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 
-public abstract class AbstractOperatorNodePushable implements IOperatorNodePushable {
+public abstract class AbstractOperatorNodePushable implements IOperatorNodePushable, IIntrospectingOperator {
     @Override
     public String getDisplayName() {
         return toString();
     }
+
+    @Override
+    public void setOperatorStats(IOperatorStats stats) {
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
index 8051305..123e765 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
@@ -108,9 +108,10 @@
      *
      * @param pid
      * @param writer
+     * @return number of bytes spilled
      * @throws HyracksDataException
      */
-    void flushPartition(int pid, IFrameWriter writer) throws HyracksDataException;
+    int flushPartition(int pid, IFrameWriter writer) throws HyracksDataException;
 
     /**
      * Clear the memory occupation of the particular partition.
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
index d3d06cb..f80efd6 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
@@ -281,17 +281,20 @@
     }
 
     @Override
-    public void flushPartition(int pid, IFrameWriter writer) throws HyracksDataException {
+    public int flushPartition(int pid, IFrameWriter writer) throws HyracksDataException {
         IFrameBufferManager partition = partitionArray[pid];
+        int written = 0;
         if (partition != null && getNumTuples(pid) > 0) {
             for (int i = 0; i < partition.getNumFrames(); ++i) {
                 partition.getFrame(i, tempInfo);
                 tempInfo.getBuffer().position(tempInfo.getStartOffset());
                 tempInfo.getBuffer().limit(tempInfo.getStartOffset() + tempInfo.getLength());
+                int sz = tempInfo.getLength();
                 writer.nextFrame(tempInfo.getBuffer());
+                written += sz;
             }
         }
-
+        return written;
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
index c545e7d..815536b 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
@@ -37,7 +37,7 @@
 import org.apache.hyracks.dataflow.std.sort.AbstractSorterOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.sort.Algorithm;
 import org.apache.hyracks.dataflow.std.sort.IRunGenerator;
-import org.apache.hyracks.dataflow.std.sort.TimedRunGenerator;
+import org.apache.hyracks.dataflow.std.sort.ProfiledRunGenerator;
 
 /**
  * This Operator pushes group-by aggregation into the external sort.
@@ -148,7 +148,8 @@
                         recordDescriptorProvider.getInputRecordDescriptor(this.getActivityId(), 0), framesLimit,
                         groupFields, keyNormalizerFactories, comparatorFactories, partialAggregatorFactory,
                         partialAggRecordDesc, ALG);
-                return profile ? TimedRunGenerator.time(runGen, ctx, "GroupBy (Sort Runs)") : runGen;
+                return profile ? ProfiledRunGenerator.time(runGen, ctx, "GroupBy (Sort Runs)", this.getActivityId())
+                        : runGen;
             }
         };
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
index 7a9bb25..b732cd4 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
@@ -34,6 +34,7 @@
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 import org.apache.hyracks.api.util.CleanupUtils;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -91,6 +92,7 @@
     // corresponding function signature.
     private final TuplePointer tempPtr = new TuplePointer();
     private int[] probePSizeInTups;
+    private IOperatorStats stats = null;
 
     public OptimizedHybridHashJoin(IHyracksJobletContext jobletCtx, int memSizeInFrames, int numOfPartitions,
             String probeRelName, String buildRelName, RecordDescriptor probeRd, RecordDescriptor buildRd,
@@ -183,7 +185,10 @@
 
     private void spillPartition(int pid) throws HyracksDataException {
         RunFileWriter writer = getSpillWriterOrCreateNewOneIfNotExist(buildRFWriters, buildRelName, pid);
-        bufferManager.flushPartition(pid, writer);
+        int spilt = bufferManager.flushPartition(pid, writer);
+        if (stats != null) {
+            stats.getBytesWritten().update(spilt);
+        }
         bufferManager.clearPartition(pid);
         spilledStatus.set(pid);
     }
@@ -261,8 +266,12 @@
             for (int pid = spilledStatus.nextSetBit(0); pid >= 0 && pid < numOfPartitions; pid =
                     spilledStatus.nextSetBit(pid + 1)) {
                 if (bufferManager.getNumTuples(pid) > 0) {
-                    bufferManager.flushPartition(pid,
+                    int spilt = bufferManager.flushPartition(pid,
                             getSpillWriterOrCreateNewOneIfNotExist(runFileWriters, refName, pid));
+                    if (stats != null) {
+                        stats.getBytesWritten().update(spilt);
+
+                    }
                     bufferManager.clearPartition(pid);
                 }
             }
@@ -417,6 +426,10 @@
                 reloadBuffer = new VSizeFrame(jobletCtx);
             }
             while (r.nextFrame(reloadBuffer)) {
+                if (stats != null) {
+                    //TODO: be certain it is the case this is actually eagerly read
+                    stats.getBytesRead().update(reloadBuffer.getBuffer().limit());
+                }
                 accessorBuild.reset(reloadBuffer.getBuffer());
                 for (int tid = 0; tid < accessorBuild.getTupleCount(); tid++) {
                     if (!bufferManager.insertTuple(pid, accessorBuild, tid, tempPtr)) {
@@ -520,7 +533,10 @@
             if (victim >= 0 && bufferManager.getPhysicalSize(victim) >= recordSize) {
                 RunFileWriter runFileWriter =
                         getSpillWriterOrCreateNewOneIfNotExist(probeRFWriters, probeRelName, victim);
-                bufferManager.flushPartition(victim, runFileWriter);
+                int spilt = bufferManager.flushPartition(victim, runFileWriter);
+                if (stats != null) {
+                    stats.getBytesWritten().update(spilt);
+                }
                 bufferManager.clearPartition(victim);
                 if (!bufferManager.insertTuple(pid, accessorProbe, tupleId, tempPtr)) {
                     // This should not happen if the size calculations are correct, just not to let the query fail.
@@ -543,6 +559,9 @@
 
             throw new HyracksDataException("The given tuple is too big");
         }
+        if (stats != null) {
+            stats.getBytesWritten().update(bigFrameAppender.getBuffer().limit());
+        }
         bigFrameAppender.write(runFileWriter, true);
     }
 
@@ -613,4 +632,8 @@
         }
         this.isReversed = reversed;
     }
+
+    public void setOperatorStats(IOperatorStats stats) {
+        this.stats = stats;
+    }
 }
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 555e8fb..e04eebe 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
@@ -45,6 +45,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
 import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.NoOpOperatorStats;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -288,6 +290,7 @@
                         new FieldHashPartitionComputerFamily(buildKeys, buildHashFunctionFactories)
                                 .createPartitioner(INIT_SEED);
                 boolean failed = false;
+                IOperatorStats stats = new NoOpOperatorStats();
 
                 @Override
                 public void open() throws HyracksDataException {
@@ -300,6 +303,7 @@
                     state.hybridHJ = new OptimizedHybridHashJoin(ctx.getJobletContext(), state.memForJoin,
                             state.numOfPartitions, PROBE_REL, BUILD_REL, probeRd, buildRd, probeHpc, buildHpc,
                             probePredEval, buildPredEval, isLeftOuter, nonMatchWriterFactories);
+                    state.hybridHJ.setOperatorStats(stats);
 
                     state.hybridHJ.initBuild();
                     if (LOGGER.isTraceEnabled()) {
@@ -341,6 +345,11 @@
                     return "Hybrid Hash Join: Build";
                 }
 
+                @Override
+                public void setOperatorStats(IOperatorStats stats) {
+                    this.stats = stats;
+                }
+
             };
         }
     }
@@ -394,6 +403,7 @@
                 private FrameTupleAppender nullResultAppender = null;
                 private FrameTupleAccessor probeTupleAccessor;
                 private boolean failed = false;
+                IOperatorStats stats = null;
 
                 @Override
                 public void open() throws HyracksDataException {
@@ -402,6 +412,7 @@
 
                     writer.open();
                     state.hybridHJ.initProbe(probComp);
+                    state.hybridHJ.setOperatorStats(stats);
 
                     if (LOGGER.isDebugEnabled()) {
                         LOGGER.debug("OptimizedHybridHashJoin is starting the probe phase.");
@@ -485,6 +496,11 @@
                     }
                 }
 
+                @Override
+                public void setOperatorStats(IOperatorStats stats) {
+                    this.stats = stats;
+                }
+
                 private void logProbeComplete() {
                     if (LOGGER.isDebugEnabled()) {
                         LOGGER.debug("OptimizedHybridHashJoin closed its probe phase");
@@ -505,6 +521,9 @@
                     long buildPartSize = (long) Math.ceil((double) buildSideReader.getFileSize() / (double) frameSize);
                     long probePartSize = (long) Math.ceil((double) probeSideReader.getFileSize() / (double) frameSize);
                     int beforeMax = Math.max(buildSizeInTuple, probeSizeInTuple);
+                    if (stats.getLevel().get() < level) {
+                        stats.getLevel().set(level);
+                    }
 
                     if (LOGGER.isDebugEnabled()) {
                         LOGGER.debug("\n>>>Joining Partition Pairs (thread_id " + Thread.currentThread().getId()
@@ -758,6 +777,7 @@
                         bReader.open();
                         rPartbuff.reset();
                         while (bReader.nextFrame(rPartbuff)) {
+                            stats.getBytesRead().update(rPartbuff.getBuffer().limit());
                             // We need to allocate a copyBuffer, because this buffer gets added to the buffers list
                             // in the InMemoryHashJoin.
                             ByteBuffer copyBuffer = bufferManager.acquireFrame(rPartbuff.getFrameSize());
@@ -786,6 +806,9 @@
                         rPartbuff.reset();
                         try {
                             while (pReader.nextFrame(rPartbuff)) {
+                                if (stats != null) {
+                                    stats.getBytesRead().update(rPartbuff.getBuffer().limit());
+                                }
                                 joiner.join(rPartbuff.getBuffer(), writer);
                                 rPartbuff.reset();
                             }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
index 654f3a3..ebe871b 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
@@ -82,7 +82,8 @@
                 final boolean profile = ctx.getJobFlags().contains(JobFlag.PROFILE_RUNTIME);
                 IRunGenerator runGen = new ExternalSortRunGenerator(ctx, sortFields, keyNormalizerFactories,
                         comparatorFactories, outRecDescs[0], alg, policy, framesLimit, outputLimit);
-                return profile ? TimedRunGenerator.time(runGen, ctx, "ExternalSort(Sort)") : runGen;
+                return profile ? ProfiledRunGenerator.time(runGen, ctx, "ExternalSort(Sort)", this.getActivityId())
+                        : runGen;
             }
         };
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ProfiledRunGenerator.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ProfiledRunGenerator.java
new file mode 100644
index 0000000..5cc1882
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ProfiledRunGenerator.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.dataflow.std.sort;
+
+import java.util.List;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.ActivityId;
+import org.apache.hyracks.api.dataflow.ProfiledFrameWriter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.IStatsCollector;
+import org.apache.hyracks.api.job.profiling.OperatorStats;
+import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
+
+public class ProfiledRunGenerator extends ProfiledFrameWriter implements IRunGenerator {
+
+    private final IRunGenerator runGenerator;
+
+    private ProfiledRunGenerator(IRunGenerator runGenerator, IStatsCollector collector, String name,
+            IOperatorStats stats, ActivityId root) {
+        super(runGenerator, collector, name, stats, null);
+        this.runGenerator = runGenerator;
+    }
+
+    @Override
+    public List<GeneratedRunFileReader> getRuns() {
+        return runGenerator.getRuns();
+    }
+
+    @Override
+    public ISorter getSorter() {
+        return runGenerator.getSorter();
+    }
+
+    public static IRunGenerator time(IRunGenerator runGenerator, IHyracksTaskContext ctx, String name, ActivityId root)
+            throws HyracksDataException {
+        if (!(runGenerator instanceof ProfiledRunGenerator)) {
+            String statName = root.toString() + " - " + name;
+            IStatsCollector statsCollector = ctx.getStatsCollector();
+            IOperatorStats stats = new OperatorStats(statName);
+            statsCollector.add(stats);
+            return new ProfiledRunGenerator(runGenerator, ctx.getStatsCollector(), name, stats, root);
+        }
+        return runGenerator;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TimedRunGenerator.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TimedRunGenerator.java
deleted file mode 100644
index b3a4aee..0000000
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TimedRunGenerator.java
+++ /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.
- */
-package org.apache.hyracks.dataflow.std.sort;
-
-import java.util.List;
-
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.TimedFrameWriter;
-import org.apache.hyracks.api.job.profiling.IStatsCollector;
-import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
-
-public class TimedRunGenerator extends TimedFrameWriter implements IRunGenerator {
-
-    private final IRunGenerator runGenerator;
-
-    private TimedRunGenerator(IRunGenerator runGenerator, IStatsCollector collector, String name) {
-        super(runGenerator, collector, name);
-        this.runGenerator = runGenerator;
-    }
-
-    @Override
-    public List<GeneratedRunFileReader> getRuns() {
-        return runGenerator.getRuns();
-    }
-
-    @Override
-    public ISorter getSorter() {
-        return runGenerator.getSorter();
-    }
-
-    public static IRunGenerator time(IRunGenerator runGenerator, IHyracksTaskContext ctx, String name) {
-        return runGenerator instanceof TimedRunGenerator ? runGenerator
-                : new TimedRunGenerator(runGenerator, ctx.getStatsCollector(), name);
-    }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java
index b7ff530..2322910 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
 import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
@@ -65,8 +66,13 @@
                 final boolean profile = ctx.getJobFlags().contains(JobFlag.PROFILE_RUNTIME);
                 IRunGenerator runGen = new HybridTopKSortRunGenerator(ctx, framesLimit, topK, sortFields,
                         keyNormalizerFactories, comparatorFactories, outRecDescs[0]);
-                return profile ? TimedRunGenerator.time(runGen, ctx, "TopKSort (Sort)") : runGen;
-
+                try {
+                    return profile ? ProfiledRunGenerator.time(runGen, ctx, "TopKSort (Sort)", this.getActivityId())
+                            : runGen;
+                } catch (HyracksDataException e) {
+                    e.printStackTrace();
+                }
+                return null;
             }
         };
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
index bcd71a3..c3bc5ca 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
@@ -49,6 +49,6 @@
                 getVirtualBufferCacheProvider(), SynchronousSchedulerProvider.INSTANCE, MERGE_POLICY_FACTORY,
                 MERGE_POLICY_PROPERTIES, DURABLE, bloomFilterKeyFields,
                 LSMTreeOperatorTestHelper.DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, true, btreefields,
-                NoOpCompressorDecompressorFactory.INSTANCE, bloomFilterKeyFields != null, null, null);
+                NoOpCompressorDecompressorFactory.INSTANCE, bloomFilterKeyFields != null, null, null, false);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java
index 4732d71..1f5b441 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java
@@ -43,7 +43,7 @@
 //Based in part on LoggingHandler from Netty
 public class CLFLogger extends ChannelDuplexHandler {
 
-    private static final Logger accessLogger = LogManager.getLogger();
+    private static final Logger LOGGER = LogManager.getLogger();
     private static final Level ACCESS_LOG_LEVEL = Level.forName("ACCESS", 550);
     private static final DateTimeFormatter DATE_TIME_FORMATTER =
             DateTimeFormatter.ofPattern("dd/MMM/yyyy:HH:mm:ss Z").withZone(ZoneId.systemDefault());
@@ -66,9 +66,14 @@
     public void channelRead(ChannelHandlerContext ctx, Object msg) {
         if (msg instanceof HttpRequest) {
             HttpRequest req = (HttpRequest) msg;
-            clientIp = ((NioSocketChannel) ctx.channel()).remoteAddress().getAddress().toString().substring(1);
+            try {
+                clientIp = ((NioSocketChannel) ctx.channel()).remoteAddress().getAddress().toString().substring(1);
+            } catch (Exception e) {
+                LOGGER.debug("ignoring {} obtaining client ip for {}", e, ctx.channel());
+                clientIp = "-";
+            }
             requestTime = Instant.now();
-            reqLine = req.method().toString() + " " + req.getUri() + " " + req.getProtocolVersion().toString();
+            reqLine = req.method().toString() + " " + req.uri() + " " + req.protocolVersion();
             userAgentRef = headerValueOrDash("Referer", req) + " " + headerValueOrDash("User-Agent", req);
             lastChunk = false;
         }
@@ -116,7 +121,7 @@
     }
 
     private void printAndPrepare() {
-        if (!accessLogger.isEnabled(ACCESS_LOG_LEVEL)) {
+        if (!LOGGER.isEnabled(ACCESS_LOG_LEVEL)) {
             return;
         }
         logLineBuilder.append(clientIp);
@@ -131,7 +136,7 @@
         logLineBuilder.append(" ").append(statusCode);
         logLineBuilder.append(" ").append(respSize);
         logLineBuilder.append(" ").append(userAgentRef);
-        accessLogger.log(ACCESS_LOG_LEVEL, logLineBuilder);
+        LOGGER.log(ACCESS_LOG_LEVEL, logLineBuilder);
         respSize = 0;
         logLineBuilder.setLength(0);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksClientInterfaceRemoteProxy.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksClientInterfaceRemoteProxy.java
index 82d7a3b..3b59a70 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksClientInterfaceRemoteProxy.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksClientInterfaceRemoteProxy.java
@@ -35,6 +35,7 @@
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobInfo;
 import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 import org.apache.hyracks.api.topology.ClusterTopology;
 import org.apache.hyracks.ipc.api.IIPCHandle;
 import org.apache.hyracks.ipc.api.RPCInterface;
@@ -130,6 +131,13 @@
     }
 
     @Override
+    public List<IOperatorStats> waitForCompletion(JobId jobId, List<String> statOperatorNames) throws Exception {
+        HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf =
+                new HyracksClientInterfaceFunctions.WaitForCompletionFunction(jobId, statOperatorNames);
+        return (List<IOperatorStats>) rpci.call(ipcHandle, wfcf);
+    }
+
+    @Override
     public Map<String, NodeControllerInfo> getNodeControllersInfo() throws Exception {
         HyracksClientInterfaceFunctions.GetNodeControllersInfoFunction gncif =
                 new HyracksClientInterfaceFunctions.GetNodeControllersInfoFunction();
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksConnection.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksConnection.java
index 9351348..1b6d8af 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksConnection.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hyracks.ipc.impl;
 
+import java.io.Closeable;
 import java.io.File;
 import java.net.InetSocketAddress;
 import java.net.URL;
@@ -52,6 +53,7 @@
 import org.apache.hyracks.api.job.JobInfo;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 import org.apache.hyracks.api.network.ISocketChannelFactory;
 import org.apache.hyracks.api.topology.ClusterTopology;
 import org.apache.hyracks.api.util.InvokeUtil;
@@ -70,7 +72,7 @@
  *
  * @author vinayakb
  */
-public final class HyracksConnection implements IHyracksClientConnection {
+public final class HyracksConnection implements Closeable, IHyracksClientConnection {
 
     private static final Logger LOGGER = LogManager.getLogger();
 
@@ -123,6 +125,11 @@
     }
 
     @Override
+    public void close() {
+        ipc.stop();
+    }
+
+    @Override
     public JobStatus getJobStatus(JobId jobId) throws Exception {
         return hci.getJobStatus(jobId);
     }
@@ -196,6 +203,17 @@
     }
 
     @Override
+    public List<IOperatorStats> waitForCompletion(JobId jobId, List<String> statOperatorNames) throws Exception {
+        try {
+            return hci.waitForCompletion(jobId, statOperatorNames);
+        } catch (InterruptedException e) {
+            // Cancels an on-going job if the current thread gets interrupted.
+            cancelJob(jobId);
+            throw e;
+        }
+    }
+
+    @Override
     public Map<String, NodeControllerInfo> getNodeControllerInfos() throws HyracksException {
         try {
             return hci.getNodeControllersInfo();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.java
new file mode 100644
index 0000000..f436636
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.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.storage.am.btree.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.DiskBTreePointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+
+/**
+ * Allows stateful {@link DiskBTreePointSearchCursor} to resume the search without closing and reopening the cursor
+ * Note: upon closing a cursor, ensure the search state is cleared
+ */
+public interface IDiskBTreeStatefulPointSearchCursor {
+    int getLastPageId();
+
+    void setCursorToNextKey(ISearchPredicate searchPred) throws HyracksDataException;
+
+    ITreeIndexFrame getFrame();
+}
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 2455625..0ab88a5 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
@@ -29,10 +29,12 @@
 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.impls.DefaultTupleProjectorFactory;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
 
 public class BTreeSearchOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
 
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 2L;
 
     protected final int[] lowKeyFields;
     protected final int[] highKeyFields;
@@ -52,6 +54,7 @@
     protected byte[] searchCallbackProceedResultTrueValue;
     protected final ITupleFilterFactory tupleFilterFactory;
     protected final long outputLimit;
+    protected final ITupleProjectorFactory tupleProjectorFactory;
 
     public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
             int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
@@ -61,7 +64,8 @@
             IMissingWriterFactory nonFilterWriterFactory) {
         this(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
-                maxFilterFieldIndexes, appendIndexFilter, nonFilterWriterFactory, null, -1, false, null, null);
+                maxFilterFieldIndexes, appendIndexFilter, nonFilterWriterFactory, null, -1, false, null, null,
+                DefaultTupleProjectorFactory.INSTANCE);
     }
 
     public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
@@ -71,7 +75,7 @@
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter,
             IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFilterFactory, long outputLimit,
             boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
-            byte[] searchCallbackProceedResultTrueValue) {
+            byte[] searchCallbackProceedResultTrueValue, ITupleProjectorFactory tupleProjectorFactory) {
         super(spec, 1, 1);
         this.indexHelperFactory = indexHelperFactory;
         this.retainInput = retainInput;
@@ -92,6 +96,7 @@
         this.appendOpCallbackProceedResult = appendOpCallbackProceedResult;
         this.searchCallbackProceedResultFalseValue = searchCallbackProceedResultFalseValue;
         this.searchCallbackProceedResultTrueValue = searchCallbackProceedResultTrueValue;
+        this.tupleProjectorFactory = tupleProjectorFactory;
     }
 
     @Override
@@ -102,7 +107,7 @@
                 lowKeyInclusive, highKeyInclusive, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
                 nonFilterWriterFactory, tupleFilterFactory, outputLimit, appendOpCallbackProceedResult,
-                searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
+                searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue, tupleProjectorFactory);
     }
 
     @Override
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 fb331bf..a56e305 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
@@ -22,6 +22,7 @@
 import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.HyracksConstants;
 import org.apache.hyracks.dataflow.common.data.accessors.PermutingFrameTupleReference;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
@@ -30,9 +31,11 @@
 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.impls.DefaultTupleProjectorFactory;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
 
 public class BTreeSearchOperatorNodePushable extends IndexSearchOperatorNodePushable {
     protected final boolean lowKeyInclusive;
@@ -52,7 +55,7 @@
         this(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
                 minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing,
                 nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter, nonFilterWriterFactory, null, -1,
-                false, null, null);
+                false, null, null, DefaultTupleProjectorFactory.INSTANCE);
     }
 
     public BTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
@@ -62,11 +65,12 @@
             ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
             IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFilterFactory, long outputLimit,
             boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
-            byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
+            byte[] searchCallbackProceedResultTrueValue, ITupleProjectorFactory projectorFactory)
+            throws HyracksDataException {
         super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
                 retainInput, retainMissing, nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter,
                 nonFilterWriterFactory, tupleFilterFactory, outputLimit, appendOpCallbackProceedResult,
-                searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
+                searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue, projectorFactory);
         this.lowKeyInclusive = lowKeyInclusive;
         this.highKeyInclusive = highKeyInclusive;
         if (lowKeyFields != null && lowKeyFields.length > 0) {
@@ -111,7 +115,8 @@
 
     @Override
     protected void addAdditionalIndexAccessorParams(IIndexAccessParameters iap) throws HyracksDataException {
-        // No additional parameters are required for the B+Tree search case
+        //Set tuple projector to get the information about the pushed down value accesses (if supported by the index)
+        iap.getParameters().put(HyracksConstants.TUPLE_PROJECTOR, tupleProjector);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
index 78faaff..40b2f5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
@@ -42,7 +42,6 @@
 import org.apache.hyracks.storage.am.btree.impls.BTreeOpContext.PageValidationInfo;
 import org.apache.hyracks.storage.am.common.api.IBTreeIndexTupleReference;
 import org.apache.hyracks.storage.am.common.api.IPageManager;
-import org.apache.hyracks.storage.am.common.api.ISplitKey;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
@@ -51,7 +50,6 @@
 import org.apache.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
 import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
-import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
 import org.apache.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
@@ -64,17 +62,13 @@
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
 import org.apache.hyracks.storage.common.NoOpIndexCursorStats;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
-import org.apache.hyracks.util.JSONUtil;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
 public class BTree extends AbstractTreeIndex {
 
     public static final float DEFAULT_FILL_FACTOR = 0.7f;
@@ -85,7 +79,7 @@
 
     private final AtomicInteger smoCounter;
     private final ReadWriteLock treeLatch;
-    private final int maxTupleSize;
+    protected final int maxTupleSize;
 
     public BTree(IBufferCache bufferCache, IPageManager freePageManager, ITreeIndexFrameFactory interiorFrameFactory,
             ITreeIndexFrameFactory leafFrameFactory, IBinaryComparatorFactory[] cmpFactories, int fieldCount,
@@ -886,13 +880,13 @@
         }
 
         @Override
-        public BTreeRangeSearchCursor createSearchCursor(boolean exclusive) {
+        public ITreeIndexCursor createSearchCursor(boolean exclusive) {
             IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
             return new BTreeRangeSearchCursor(leafFrame, exclusive, (IIndexCursorStats) iap.getParameters()
                     .getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE));
         }
 
-        public BTreeRangeSearchCursor createPointCursor(boolean exclusive, boolean stateful) {
+        public ITreeIndexCursor createPointCursor(boolean exclusive, boolean stateful) {
             return createSearchCursor(exclusive);
         }
 
@@ -1004,220 +998,7 @@
     @Override
     public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
             boolean checkIfEmptyIndex, IPageWriteCallback callback) throws HyracksDataException {
-        return new BTreeBulkLoader(fillFactor, verifyInput, callback);
-    }
-
-    public class BTreeBulkLoader extends AbstractTreeIndex.AbstractTreeIndexBulkLoader {
-        protected final ISplitKey splitKey;
-        protected final boolean verifyInput;
-
-        public BTreeBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback)
-                throws HyracksDataException {
-            super(fillFactor, callback);
-            this.verifyInput = verifyInput;
-            splitKey = new BTreeSplitKey(leafFrame.getTupleWriter().createTupleReference());
-            splitKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
-        }
-
-        @Override
-        public void add(ITupleReference tuple) throws HyracksDataException {
-            try {
-                int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
-                        interiorFrame.getBytesRequiredToWriteTuple(tuple));
-                NodeFrontier leafFrontier = nodeFrontiers.get(0);
-                int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
-                int spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
-
-                // try to free space by compression
-                if (spaceUsed + spaceNeeded > leafMaxBytes) {
-                    leafFrame.compress();
-                    spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
-                }
-                //full, allocate new page
-                if (spaceUsed + spaceNeeded > leafMaxBytes) {
-                    if (leafFrame.getTupleCount() == 0) {
-                        bufferCache.returnPage(leafFrontier.page, false);
-                    } else {
-                        leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
-                        if (verifyInput) {
-                            verifyInputTuple(tuple, leafFrontier.lastTuple);
-                        }
-                        int splitKeySize = tupleWriter.bytesRequired(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount());
-                        splitKey.initData(splitKeySize);
-                        tupleWriter.writeTupleFields(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount(),
-                                splitKey.getBuffer().array(), 0);
-                        splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
-                        splitKey.setLeftPage(leafFrontier.pageId);
-
-                        propagateBulk(1, pagesToWrite);
-
-                        leafFrontier.pageId = freePageManager.takePage(metaFrame);
-
-                        ((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId);
-
-                        write(leafFrontier.page);
-                        for (ICachedPage c : pagesToWrite) {
-                            write(c);
-                        }
-                        pagesToWrite.clear();
-                        splitKey.setRightPage(leafFrontier.pageId);
-                    }
-                    if (tupleSize > maxTupleSize) {
-                        final long dpid = BufferedFileHandle.getDiskPageId(getFileId(), leafFrontier.pageId);
-                        // calculate required number of pages.
-                        int headerSize = Math.max(leafFrame.getPageHeaderSize(), interiorFrame.getPageHeaderSize());
-                        final int multiplier =
-                                (int) Math.ceil((double) tupleSize / (bufferCache.getPageSize() - headerSize));
-                        if (multiplier > 1) {
-                            leafFrontier.page = bufferCache.confiscateLargePage(dpid, multiplier,
-                                    freePageManager.takeBlock(metaFrame, multiplier - 1));
-                        } else {
-                            leafFrontier.page = bufferCache.confiscatePage(dpid);
-                        }
-                        leafFrame.setPage(leafFrontier.page);
-                        leafFrame.initBuffer((byte) 0);
-                        ((IBTreeLeafFrame) leafFrame).setLargeFlag(true);
-                    } else {
-                        final long dpid = BufferedFileHandle.getDiskPageId(getFileId(), leafFrontier.pageId);
-                        leafFrontier.page = bufferCache.confiscatePage(dpid);
-                        leafFrame.setPage(leafFrontier.page);
-                        leafFrame.initBuffer((byte) 0);
-                    }
-                } else {
-                    if (verifyInput && leafFrame.getTupleCount() > 0) {
-                        leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
-                        verifyInputTuple(tuple, leafFrontier.lastTuple);
-                    }
-                }
-                ((IBTreeLeafFrame) leafFrame).insertSorted(tuple);
-            } catch (HyracksDataException | RuntimeException e) {
-                logState(tuple, e);
-                handleException();
-                throw e;
-            }
-        }
-
-        protected void verifyInputTuple(ITupleReference tuple, ITupleReference prevTuple) throws HyracksDataException {
-            // New tuple should be strictly greater than last tuple.
-            int cmpResult = cmp.compare(tuple, prevTuple);
-            if (cmpResult < 0) {
-                throw HyracksDataException.create(ErrorCode.UNSORTED_LOAD_INPUT);
-            }
-            if (cmpResult == 0) {
-                throw HyracksDataException.create(ErrorCode.DUPLICATE_LOAD_INPUT);
-            }
-        }
-
-        protected void propagateBulk(int level, List<ICachedPage> pagesToWrite) throws HyracksDataException {
-            if (splitKey.getBuffer() == null) {
-                return;
-            }
-
-            if (level >= nodeFrontiers.size()) {
-                addLevel();
-            }
-
-            NodeFrontier frontier = nodeFrontiers.get(level);
-            interiorFrame.setPage(frontier.page);
-
-            ITupleReference tuple = splitKey.getTuple();
-            int tupleBytes = tupleWriter.bytesRequired(tuple, 0, cmp.getKeyFieldCount());
-            int spaceNeeded = tupleBytes + slotSize + 4;
-            if (tupleBytes > interiorFrame.getMaxTupleSize(BTree.this.bufferCache.getPageSize())) {
-                throw HyracksDataException.create(ErrorCode.RECORD_IS_TOO_LARGE, tupleBytes,
-                        interiorFrame.getMaxTupleSize(BTree.this.bufferCache.getPageSize()));
-            }
-
-            int spaceUsed = interiorFrame.getBuffer().capacity() - interiorFrame.getTotalFreeSpace();
-            if (spaceUsed + spaceNeeded > interiorMaxBytes) {
-
-                ISplitKey copyKey = splitKey.duplicate(leafFrame.getTupleWriter().createTupleReference());
-                tuple = copyKey.getTuple();
-
-                frontier.lastTuple.resetByTupleIndex(interiorFrame, interiorFrame.getTupleCount() - 1);
-                int splitKeySize = tupleWriter.bytesRequired(frontier.lastTuple, 0, cmp.getKeyFieldCount());
-                splitKey.initData(splitKeySize);
-                tupleWriter.writeTupleFields(frontier.lastTuple, 0, cmp.getKeyFieldCount(),
-                        splitKey.getBuffer().array(), 0);
-                splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
-
-                ((IBTreeInteriorFrame) interiorFrame).deleteGreatest();
-                int finalPageId = freePageManager.takePage(metaFrame);
-                frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), finalPageId));
-                pagesToWrite.add(frontier.page);
-                splitKey.setLeftPage(finalPageId);
-
-                propagateBulk(level + 1, pagesToWrite);
-                frontier.page = bufferCache.confiscatePage(BufferCache.INVALID_DPID);
-                interiorFrame.setPage(frontier.page);
-                interiorFrame.initBuffer((byte) level);
-            }
-            ((IBTreeInteriorFrame) interiorFrame).insertSorted(tuple);
-        }
-
-        private void persistFrontiers(int level, int rightPage) throws HyracksDataException {
-            if (level >= nodeFrontiers.size()) {
-                rootPage = nodeFrontiers.get(level - 1).pageId;
-                releasedLatches = true;
-                return;
-            }
-            if (level < 1) {
-                ICachedPage lastLeaf = nodeFrontiers.get(level).page;
-                int lastLeafPage = nodeFrontiers.get(level).pageId;
-                lastLeaf.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), nodeFrontiers.get(level).pageId));
-                write(lastLeaf);
-                nodeFrontiers.get(level).page = null;
-                persistFrontiers(level + 1, lastLeafPage);
-                return;
-            }
-            NodeFrontier frontier = nodeFrontiers.get(level);
-            interiorFrame.setPage(frontier.page);
-            //just finalize = the layer right above the leaves has correct righthand pointers already
-            if (rightPage < 0) {
-                throw new HyracksDataException(
-                        "Error in index creation. Internal node appears to have no rightmost guide");
-            }
-            ((IBTreeInteriorFrame) interiorFrame).setRightmostChildPageId(rightPage);
-            int finalPageId = freePageManager.takePage(metaFrame);
-            frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), finalPageId));
-            write(frontier.page);
-            frontier.pageId = finalPageId;
-            persistFrontiers(level + 1, finalPageId);
-        }
-
-        @Override
-        public void end() throws HyracksDataException {
-            try {
-                persistFrontiers(0, -1);
-                super.end();
-            } catch (HyracksDataException | RuntimeException e) {
-                handleException();
-                throw e;
-            }
-        }
-
-        @Override
-        public void abort() throws HyracksDataException {
-            super.handleException();
-        }
-
-        private void logState(ITupleReference tuple, Exception e) {
-            try {
-                ObjectNode state = JSONUtil.createObject();
-                state.set("leafFrame", leafFrame.getState());
-                state.set("interiorFrame", interiorFrame.getState());
-                int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
-                        interiorFrame.getBytesRequiredToWriteTuple(tuple));
-                state.put("tupleSize", tupleSize);
-                state.put("spaceNeeded", tupleWriter.bytesRequired(tuple) + slotSize);
-                state.put("spaceUsed", leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace());
-                state.put("leafMaxBytes", leafMaxBytes);
-                state.put("maxTupleSize", maxTupleSize);
-                LOGGER.error("failed to add tuple {}", state, e);
-            } catch (Throwable t) {
-                e.addSuppressed(t);
-            }
-        }
+        return new BTreeNSMBulkLoader(fillFactor, verifyInput, callback, this);
     }
 
     @SuppressWarnings("rawtypes")
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java
new file mode 100644
index 0000000..04c84e1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java
@@ -0,0 +1,279 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree.impls;
+
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
+import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.common.api.ISplitKey;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndexBulkLoader;
+import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+import org.apache.hyracks.util.JSONUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class BTreeNSMBulkLoader extends AbstractTreeIndexBulkLoader {
+    private static final Logger LOGGER = LogManager.getLogger();
+    protected final ISplitKey splitKey;
+    protected final boolean verifyInput;
+    private final int maxTupleSize;
+
+    public BTreeNSMBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback, ITreeIndex index)
+            throws HyracksDataException {
+        this(fillFactor, verifyInput, callback, index, index.getLeafFrameFactory().createFrame());
+    }
+
+    protected BTreeNSMBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback, ITreeIndex index,
+            ITreeIndexFrame leafFrame) throws HyracksDataException {
+        super(fillFactor, callback, index, leafFrame);
+        this.verifyInput = verifyInput;
+        splitKey = new BTreeSplitKey(tupleWriter.createTupleReference());
+        splitKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
+        maxTupleSize = ((BTree) index).maxTupleSize;
+    }
+
+    @Override
+    public void add(ITupleReference tuple) throws HyracksDataException {
+        try {
+            int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
+                    interiorFrame.getBytesRequiredToWriteTuple(tuple));
+            NodeFrontier leafFrontier = nodeFrontiers.get(0);
+            int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
+            int spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
+
+            // try to free space by compression
+            if (spaceUsed + spaceNeeded > leafMaxBytes) {
+                leafFrame.compress();
+                spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
+            }
+            //full, allocate new page
+            if (spaceUsed + spaceNeeded > leafMaxBytes) {
+                if (leafFrame.getTupleCount() == 0) {
+                    //The current page is empty. Return it.
+                    bufferCache.returnPage(leafFrontier.page, false);
+                } else {
+                    leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+                    if (verifyInput) {
+                        verifyInputTuple(tuple, leafFrontier.lastTuple);
+                    }
+                    //The current page is not empty. Write it.
+                    writeFullLeafPage();
+                }
+                if (tupleSize > maxTupleSize) {
+                    //We need a large page
+                    final long dpid = BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId);
+                    // calculate required number of pages.
+                    int headerSize = Math.max(leafFrame.getPageHeaderSize(), interiorFrame.getPageHeaderSize());
+                    final int multiplier =
+                            (int) Math.ceil((double) tupleSize / (bufferCache.getPageSize() - headerSize));
+                    if (multiplier > 1) {
+                        leafFrontier.page = bufferCache.confiscateLargePage(dpid, multiplier,
+                                freePageManager.takeBlock(metaFrame, multiplier - 1));
+                    } else {
+                        leafFrontier.page = bufferCache.confiscatePage(dpid);
+                    }
+                    leafFrame.setPage(leafFrontier.page);
+                    leafFrame.initBuffer((byte) 0);
+                    ((IBTreeLeafFrame) leafFrame).setLargeFlag(true);
+                } else {
+                    //allocate a new page
+                    confiscateNewLeafPage();
+                }
+            } else {
+                if (verifyInput && leafFrame.getTupleCount() > 0) {
+                    leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+                    verifyInputTuple(tuple, leafFrontier.lastTuple);
+                }
+            }
+            ((IBTreeLeafFrame) leafFrame).insertSorted(tuple);
+        } catch (HyracksDataException | RuntimeException e) {
+            logState(tuple, e);
+            handleException();
+            throw e;
+        }
+    }
+
+    protected void verifyInputTuple(ITupleReference tuple, ITupleReference prevTuple) throws HyracksDataException {
+        // New tuple should be strictly greater than last tuple.
+        int cmpResult = cmp.compare(tuple, prevTuple);
+        if (cmpResult < 0) {
+            throw HyracksDataException.create(ErrorCode.UNSORTED_LOAD_INPUT);
+        }
+        if (cmpResult == 0) {
+            throw HyracksDataException.create(ErrorCode.DUPLICATE_LOAD_INPUT);
+        }
+    }
+
+    protected void propagateBulk(int level, List<ICachedPage> pagesToWrite) throws HyracksDataException {
+        if (splitKey.getBuffer() == null) {
+            return;
+        }
+
+        if (level >= nodeFrontiers.size()) {
+            addLevel();
+        }
+
+        NodeFrontier frontier = nodeFrontiers.get(level);
+        interiorFrame.setPage(frontier.page);
+
+        ITupleReference tuple = splitKey.getTuple();
+        int tupleBytes = tupleWriter.bytesRequired(tuple, 0, cmp.getKeyFieldCount());
+        int spaceNeeded = tupleBytes + slotSize + 4;
+        if (tupleBytes > interiorFrame.getMaxTupleSize(bufferCache.getPageSize())) {
+            throw HyracksDataException.create(ErrorCode.RECORD_IS_TOO_LARGE, tupleBytes,
+                    interiorFrame.getMaxTupleSize(bufferCache.getPageSize()));
+        }
+
+        int spaceUsed = interiorFrame.getBuffer().capacity() - interiorFrame.getTotalFreeSpace();
+        if (spaceUsed + spaceNeeded > interiorMaxBytes) {
+            ISplitKey copyKey = splitKey.duplicate(tupleWriter.createTupleReference());
+            tuple = copyKey.getTuple();
+
+            frontier.lastTuple.resetByTupleIndex(interiorFrame, interiorFrame.getTupleCount() - 1);
+            int splitKeySize = tupleWriter.bytesRequired(frontier.lastTuple, 0, cmp.getKeyFieldCount());
+            splitKey.initData(splitKeySize);
+            tupleWriter.writeTupleFields(frontier.lastTuple, 0, cmp.getKeyFieldCount(), splitKey.getBuffer().array(),
+                    0);
+            splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+
+            ((IBTreeInteriorFrame) interiorFrame).deleteGreatest();
+            int finalPageId = freePageManager.takePage(metaFrame);
+            frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, finalPageId));
+            pagesToWrite.add(frontier.page);
+            splitKey.setLeftPage(finalPageId);
+
+            propagateBulk(level + 1, pagesToWrite);
+            frontier.page = bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
+            interiorFrame.setPage(frontier.page);
+            interiorFrame.initBuffer((byte) level);
+        }
+        ((IBTreeInteriorFrame) interiorFrame).insertSorted(tuple);
+    }
+
+    private void persistFrontiers(int level, int rightPage) throws HyracksDataException {
+        if (level >= nodeFrontiers.size()) {
+            setRootPageId(nodeFrontiers.get(level - 1).pageId);
+            releasedLatches = true;
+            return;
+        }
+        if (level < 1) {
+            ICachedPage lastLeaf = nodeFrontiers.get(level).page;
+            int lastLeafPage = nodeFrontiers.get(level).pageId;
+            lastLeaf.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, nodeFrontiers.get(level).pageId));
+            writeLastLeaf(lastLeaf);
+            nodeFrontiers.get(level).page = null;
+            persistFrontiers(level + 1, lastLeafPage);
+            return;
+        }
+        NodeFrontier frontier = nodeFrontiers.get(level);
+        interiorFrame.setPage(frontier.page);
+        //just finalize = the layer right above the leaves has correct righthand pointers already
+        if (rightPage < 0) {
+            throw new HyracksDataException("Error in index creation. Internal node appears to have no rightmost guide");
+        }
+        ((IBTreeInteriorFrame) interiorFrame).setRightmostChildPageId(rightPage);
+        int finalPageId = freePageManager.takePage(metaFrame);
+        frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, finalPageId));
+        write(frontier.page);
+        frontier.pageId = finalPageId;
+        persistFrontiers(level + 1, finalPageId);
+    }
+
+    @Override
+    public void end() throws HyracksDataException {
+        try {
+            persistFrontiers(0, -1);
+            super.end();
+        } catch (HyracksDataException | RuntimeException e) {
+            handleException();
+            throw e;
+        }
+    }
+
+    @Override
+    public void abort() throws HyracksDataException {
+        super.handleException();
+    }
+
+    protected void writeFullLeafPage() throws HyracksDataException {
+        final NodeFrontier leafFrontier = nodeFrontiers.get(0);
+        leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+        final int splitKeySize = tupleWriter.bytesRequired(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount());
+        splitKey.initData(splitKeySize);
+        tupleWriter.writeTupleFields(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount(), splitKey.getBuffer().array(),
+                0);
+        splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+        splitKey.setLeftPage(leafFrontier.pageId);
+
+        propagateBulk(1, pagesToWrite);
+
+        leafFrontier.pageId = freePageManager.takePage(metaFrame);
+
+        ((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId);
+
+        write(leafFrontier.page);
+        for (ICachedPage c : pagesToWrite) {
+            write(c);
+        }
+        pagesToWrite.clear();
+        splitKey.setRightPage(leafFrontier.pageId);
+    }
+
+    protected void writeLastLeaf(ICachedPage page) throws HyracksDataException {
+        write(page);
+    }
+
+    protected final void confiscateNewLeafPage() throws HyracksDataException {
+        final NodeFrontier leafFrontier = nodeFrontiers.get(0);
+        final long dpid = BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId);
+        leafFrontier.page = bufferCache.confiscatePage(dpid);
+        leafFrame.setPage(leafFrontier.page);
+        leafFrame.initBuffer((byte) 0);
+    }
+
+    private void logState(ITupleReference tuple, Exception e) {
+        try {
+            ObjectNode state = JSONUtil.createObject();
+            state.set("leafFrame", leafFrame.getState());
+            state.set("interiorFrame", interiorFrame.getState());
+            int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
+                    interiorFrame.getBytesRequiredToWriteTuple(tuple));
+            state.put("tupleSize", tupleSize);
+            state.put("spaceNeeded", tupleWriter.bytesRequired(tuple) + slotSize);
+            state.put("spaceUsed", leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace());
+            state.put("leafMaxBytes", leafMaxBytes);
+            state.put("maxTupleSize", maxTupleSize);
+            LOGGER.error("failed to add tuple {}", state, e);
+        } catch (Throwable t) {
+            e.addSuppressed(t);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
index ae6bbaa..179f1da 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
@@ -24,8 +24,8 @@
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.util.HyracksConstants;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.btree.api.IBTreeFrame;
 import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.btree.api.IDiskBTreeStatefulPointSearchCursor;
 import org.apache.hyracks.storage.am.common.api.IPageManager;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
@@ -38,7 +38,6 @@
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
 import org.apache.hyracks.storage.common.NoOpIndexCursorStats;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
@@ -76,7 +75,8 @@
     private void search(ITreeIndexCursor cursor, ISearchPredicate searchPred, BTreeOpContext ctx)
             throws HyracksDataException {
         ctx.reset();
-        ctx.setPred((RangePredicate) searchPred);
+        RangePredicate rangePredicate = (RangePredicate) searchPred;
+        ctx.setPred(rangePredicate);
         ctx.setCursor(cursor);
         if (ctx.getPred().getLowKeyComparator() == null) {
             ctx.getPred().setLowKeyComparator(ctx.getCmp());
@@ -87,25 +87,17 @@
         cursor.setBufferCache(bufferCache);
         cursor.setFileId(getFileId());
 
-        if (cursor instanceof DiskBTreePointSearchCursor) {
-            DiskBTreePointSearchCursor pointCursor = (DiskBTreePointSearchCursor) cursor;
+        if (cursor instanceof IDiskBTreeStatefulPointSearchCursor) {
+            IDiskBTreeStatefulPointSearchCursor pointCursor = (IDiskBTreeStatefulPointSearchCursor) cursor;
             int lastPageId = pointCursor.getLastPageId();
-            if (lastPageId != BufferCache.INVALID_PAGEID) {
-                // check whether the last leaf page contains this key
-                ICachedPage lastPage =
-                        bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), lastPageId), false);
-                ctx.getLeafFrame().setPage(lastPage);
-                if (fitInPage(ctx.getPred().getLowKey(), ctx.getPred().getLowKeyComparator(), ctx.getLeafFrame())) {
-                    // use this page
-                    ctx.getCursorInitialState().setPage(lastPage);
-                    ctx.getCursorInitialState().setPageId(lastPageId);
-                    pointCursor.open(ctx.getCursorInitialState(), searchPred);
+            if (lastPageId != IBufferCache.INVALID_PAGEID) {
+                if (fitInPage(ctx.getPred().getLowKey(), ctx.getPred().getLowKeyComparator(), pointCursor.getFrame())) {
+                    pointCursor.setCursorToNextKey(searchPred);
                     return;
                 } else {
-                    // release the last page and clear the states of this cursor
+                    // release the last page, clear the states of this cursor, and close the cursor
                     // then retry the search from root to leaf
-                    bufferCache.unpin(lastPage);
-                    pointCursor.clearSearchState();
+                    cursor.close();
                 }
             }
         }
@@ -113,7 +105,7 @@
         searchDown(rootNode, rootPage, ctx, cursor);
     }
 
-    private boolean fitInPage(ITupleReference key, MultiComparator comparator, IBTreeFrame frame)
+    private boolean fitInPage(ITupleReference key, MultiComparator comparator, ITreeIndexFrame frame)
             throws HyracksDataException {
         // assume that search keys are sorted (non-decreasing)
         ITupleReference rightmostTuple = frame.getRightmostTuple();
@@ -191,14 +183,14 @@
         }
 
         @Override
-        public DiskBTreeRangeSearchCursor createSearchCursor(boolean exclusive) {
+        public ITreeIndexCursor createSearchCursor(boolean exclusive) {
             IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
             return new DiskBTreeRangeSearchCursor(leafFrame, exclusive, (IIndexCursorStats) iap.getParameters()
                     .getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE));
         }
 
         @Override
-        public BTreeRangeSearchCursor createPointCursor(boolean exclusive, boolean stateful) {
+        public ITreeIndexCursor createPointCursor(boolean exclusive, boolean stateful) {
             IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
             return new DiskBTreePointSearchCursor(leafFrame, exclusive, stateful);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
index 1bf3ecf..8fd9a96 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
@@ -21,13 +21,16 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.btree.api.IDiskBTreeStatefulPointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
 import org.apache.hyracks.storage.am.common.ophelpers.FindTupleMode;
 import org.apache.hyracks.storage.am.common.ophelpers.FindTupleNoExactMatchPolicy;
 import org.apache.hyracks.storage.common.ICursorInitialState;
 import org.apache.hyracks.storage.common.ISearchPredicate;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
-public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor {
+public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor
+        implements IDiskBTreeStatefulPointSearchCursor {
     /**
      * A stateful cursor keeps the search state (last search page Id + index) across multiple searches
      * until {@link #clearSearchState()} is called explicity
@@ -36,16 +39,14 @@
 
     private boolean nextHasBeenCalled;
 
-    private int lastPageId = BufferCache.INVALID_PAGEID;
+    private int lastPageId;
     private int lastTupleIndex = 0;
 
     public DiskBTreePointSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes, boolean stateful) {
         super(frame, exclusiveLatchNodes);
         this.stateful = stateful;
-    }
-
-    public DiskBTreePointSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes) {
-        this(frame, exclusiveLatchNodes, false);
+        lastPageId = IBufferCache.INVALID_PAGEID;
+        lastTupleIndex = 0;
     }
 
     @Override
@@ -59,6 +60,12 @@
     }
 
     @Override
+    public void doClose() throws HyracksDataException {
+        clearSearchState();
+        super.doClose();
+    }
+
+    @Override
     public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         // in case open is called multiple times without closing
         if (page != null) {
@@ -71,30 +78,10 @@
         page = initialState.getPage();
         isPageDirty = false;
         frame.setPage(page);
-
-        pred = (RangePredicate) searchPred;
-        lowKeyCmp = pred.getLowKeyComparator();
-        lowKey = pred.getLowKey();
-
-        reusablePredicate.setLowKeyComparator(originalKeyCmp);
-
-        lowKeyFtm = FindTupleMode.EXACT;
-        lowKeyFtp = FindTupleNoExactMatchPolicy.NONE;
-
-        nextHasBeenCalled = false;
-
-        // only get the low key position
-        tupleIndex = getLowKeyIndex();
-        if (stateful) {
-            lastPageId = pageId;
-            if (tupleIndex >= 0) {
-                lastTupleIndex = tupleIndex;
-            } else {
-                lastTupleIndex = -tupleIndex - 1;
-            }
-        }
+        setCursorToNextKey(searchPred);
     }
 
+    @Override
     public int getLastPageId() {
         return lastPageId;
     }
@@ -108,9 +95,38 @@
         }
     }
 
-    public void clearSearchState() {
-        this.lastPageId = BufferCache.INVALID_PAGEID;
+    @Override
+    public void setCursorToNextKey(ISearchPredicate searchPred) throws HyracksDataException {
+        pred = (RangePredicate) searchPred;
+        lowKeyCmp = pred.getLowKeyComparator();
+        lowKey = pred.getLowKey();
+
+        reusablePredicate.setLowKeyComparator(originalKeyCmp);
+
+        lowKeyFtm = FindTupleMode.EXACT;
+        lowKeyFtp = FindTupleNoExactMatchPolicy.NONE;
+
+        nextHasBeenCalled = false;
+
+        // only get the lowKey position
+        tupleIndex = getLowKeyIndex();
+        if (stateful) {
+            lastPageId = pageId;
+            if (tupleIndex >= 0) {
+                lastTupleIndex = tupleIndex;
+            } else {
+                lastTupleIndex = -tupleIndex - 1;
+            }
+        }
+    }
+
+    private void clearSearchState() {
+        this.lastPageId = IBufferCache.INVALID_PAGEID;
         this.lastTupleIndex = 0;
     }
 
+    @Override
+    public ITreeIndexFrame getFrame() {
+        return frame;
+    }
 }
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 eab8c96..d555b31 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
@@ -24,11 +24,13 @@
 
 import org.apache.hyracks.api.comm.VSizeFrame;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IIntrospectingOperator;
 import org.apache.hyracks.api.dataflow.value.IMissingWriter;
 import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.NoOpOperatorStats;
 import org.apache.hyracks.api.util.CleanupUtils;
 import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -44,6 +46,7 @@
 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.DefaultTupleProjectorFactory;
 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.ReferenceFrameTupleReference;
@@ -54,12 +57,15 @@
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.projection.ITupleProjector;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
 import org.apache.hyracks.util.IThreadStatsCollector;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
-public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable
+        implements IIntrospectingOperator {
 
     static final Logger LOGGER = LogManager.getLogger();
     protected final IHyracksTaskContext ctx;
@@ -107,6 +113,7 @@
     protected final long outputLimit;
     protected long outputCount = 0;
     protected boolean finished;
+    protected final ITupleProjector tupleProjector;
 
     // no filter and limit pushdown
     public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
@@ -116,7 +123,7 @@
             IMissingWriterFactory nonFilterWriterFactory) throws HyracksDataException {
         this(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
                 retainInput, retainMissing, nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter,
-                nonFilterWriterFactory, null, -1, false, null, null);
+                nonFilterWriterFactory, null, -1, false, null, null, DefaultTupleProjectorFactory.INSTANCE);
     }
 
     public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
@@ -125,7 +132,8 @@
             ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
             IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFactoryFactory, long outputLimit,
             boolean appendSearchCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
-            byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
+            byte[] searchCallbackProceedResultTrueValue, ITupleProjectorFactory projectorFactory)
+            throws HyracksDataException {
         this.ctx = ctx;
         this.indexHelper = indexHelperFactory.create(ctx.getJobletContext().getServiceContext(), partition);
         this.retainInput = retainInput;
@@ -154,14 +162,13 @@
         this.searchCallbackProceedResultTrueValue = searchCallbackProceedResultTrueValue;
         this.tupleFilterFactory = tupleFactoryFactory;
         this.outputLimit = outputLimit;
-
-        if (ctx != null && ctx.getStatsCollector() != null) {
-            stats = ctx.getStatsCollector().getOrAddOperatorStats(getDisplayName());
-        }
+        this.stats = new NoOpOperatorStats();
 
         if (this.tupleFilterFactory != null && this.retainMissing) {
             throw new IllegalStateException("RetainMissing with tuple filter is not supported");
         }
+
+        tupleProjector = projectorFactory.createTupleProjector(ctx);
     }
 
     protected abstract ISearchPredicate createSearchPredicate();
@@ -266,7 +273,7 @@
                 break;
             }
         }
-        stats.getTupleCounter().update(matchingTupleCount);
+        stats.getInputTupleCounter().update(matchingTupleCount);
 
         if (matchingTupleCount == 0 && retainInput && retainMissing) {
             FrameUtils.appendConcatToWriter(writer, appender, accessor, tupleIndex,
@@ -314,7 +321,8 @@
                     if (appender.getTupleCount() > 0) {
                         appender.write(writer, true);
                     }
-                    stats.getDiskIoCounter().update(ctx.getThreadStats().getPinnedPagesCount());
+                    stats.getPageReads().update(ctx.getThreadStats().getPinnedPagesCount());
+                    stats.coldReadCounter().update(ctx.getThreadStats().getColdReadCount());
                 } catch (Throwable th) { // NOSONAR Must ensure writer.fail is called.
                     // subsequently, the failure will be thrown
                     failure = th;
@@ -349,10 +357,7 @@
 
     protected void writeTupleToOutput(ITupleReference tuple) throws IOException {
         try {
-            for (int i = 0; i < tuple.getFieldCount(); i++) {
-                dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
-                tb.addFieldEndOffset();
-            }
+            tupleProjector.project(tuple, dos, tb);
         } catch (Exception e) {
             throw e;
         }
@@ -399,4 +404,9 @@
         return "Index Search";
     }
 
+    @Override
+    public void setOperatorStats(IOperatorStats stats) {
+        this.stats = stats;
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
index 81e528b..11368bf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
@@ -19,9 +19,6 @@
 
 package org.apache.hyracks.storage.am.common.impls;
 
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,17 +27,9 @@
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexMetadataFrame;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
-import org.apache.hyracks.storage.common.IIndexBulkLoader;
-import org.apache.hyracks.storage.common.MultiComparator;
 import org.apache.hyracks.storage.common.buffercache.HaltOnFailureCallback;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
-import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
-import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
-import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
-import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 
 public abstract class AbstractTreeIndex implements ITreeIndex {
@@ -229,118 +218,6 @@
         return fieldCount;
     }
 
-    public abstract class AbstractTreeIndexBulkLoader extends PageWriteFailureCallback implements IIndexBulkLoader {
-        protected final MultiComparator cmp;
-        protected final int slotSize;
-        protected final int leafMaxBytes;
-        protected final int interiorMaxBytes;
-        protected final ArrayList<NodeFrontier> nodeFrontiers = new ArrayList<>();
-        protected final ITreeIndexMetadataFrame metaFrame;
-        protected final ITreeIndexTupleWriter tupleWriter;
-        protected ITreeIndexFrame leafFrame;
-        protected ITreeIndexFrame interiorFrame;
-        // Immutable bulk loaders write their root page at page -2, as needed e.g. by append-only file systems such as
-        // HDFS.  Since loading this tree relies on the root page actually being at that point, no further inserts into
-        // that tree are allowed.  Currently, this is not enforced.
-        protected boolean releasedLatches;
-        private final IFIFOPageWriter pageWriter;
-        protected List<ICachedPage> pagesToWrite;
-        private final ICompressedPageWriter compressedPageWriter;
-
-        public AbstractTreeIndexBulkLoader(float fillFactor, IPageWriteCallback callback) throws HyracksDataException {
-            leafFrame = leafFrameFactory.createFrame();
-            interiorFrame = interiorFrameFactory.createFrame();
-            metaFrame = freePageManager.createMetadataFrame();
-
-            pageWriter = bufferCache.createFIFOWriter(callback, this);
-
-            if (!isEmptyTree(leafFrame)) {
-                throw HyracksDataException.create(ErrorCode.CANNOT_BULK_LOAD_NON_EMPTY_TREE);
-            }
-
-            this.cmp = MultiComparator.create(cmpFactories);
-
-            leafFrame.setMultiComparator(cmp);
-            interiorFrame.setMultiComparator(cmp);
-
-            tupleWriter = leafFrame.getTupleWriter();
-
-            NodeFrontier leafFrontier = new NodeFrontier(leafFrame.createTupleReference());
-            leafFrontier.pageId = freePageManager.takePage(metaFrame);
-            leafFrontier.page =
-                    bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId));
-
-            interiorFrame.setPage(leafFrontier.page);
-            interiorFrame.initBuffer((byte) 0);
-            interiorMaxBytes = (int) (interiorFrame.getBuffer().capacity() * fillFactor);
-
-            leafFrame.setPage(leafFrontier.page);
-            leafFrame.initBuffer((byte) 0);
-            leafMaxBytes = (int) (leafFrame.getBuffer().capacity() * fillFactor);
-            slotSize = leafFrame.getSlotSize();
-
-            nodeFrontiers.add(leafFrontier);
-            pagesToWrite = new ArrayList<>();
-            compressedPageWriter = bufferCache.getCompressedPageWriter(fileId);
-        }
-
-        protected void handleException() {
-            // Unlatch and unpin pages that weren't in the queue to avoid leaking memory.
-            compressedPageWriter.abort();
-            for (NodeFrontier nodeFrontier : nodeFrontiers) {
-                if (nodeFrontier != null && nodeFrontier.page != null) {
-                    ICachedPage frontierPage = nodeFrontier.page;
-                    if (frontierPage.confiscated()) {
-                        bufferCache.returnPage(frontierPage, false);
-                    }
-                }
-            }
-            for (ICachedPage pageToDiscard : pagesToWrite) {
-                if (pageToDiscard != null) {
-                    bufferCache.returnPage(pageToDiscard, false);
-                }
-            }
-            releasedLatches = true;
-        }
-
-        @Override
-        public void end() throws HyracksDataException {
-            if (hasFailed()) {
-                throw HyracksDataException.create(getFailure());
-            }
-            freePageManager.setRootPageId(rootPage);
-        }
-
-        protected void addLevel() throws HyracksDataException {
-            NodeFrontier frontier = new NodeFrontier(tupleWriter.createTupleReference());
-            frontier.page = bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
-            frontier.pageId = -1;
-            frontier.lastTuple.setFieldCount(cmp.getKeyFieldCount());
-            interiorFrame.setPage(frontier.page);
-            interiorFrame.initBuffer((byte) nodeFrontiers.size());
-            nodeFrontiers.add(frontier);
-        }
-
-        public ITreeIndexFrame getLeafFrame() {
-            return leafFrame;
-        }
-
-        public void setLeafFrame(ITreeIndexFrame leafFrame) {
-            this.leafFrame = leafFrame;
-        }
-
-        public void write(ICachedPage cPage) throws HyracksDataException {
-            compressedPageWriter.prepareWrite(cPage);
-            pageWriter.write(cPage);
-        }
-
-        @Override
-        public void force() throws HyracksDataException {
-            bufferCache.force(fileId, false);
-        }
-
-    }
-
     public IBinaryComparatorFactory[] getCmpFactories() {
         return cmpFactories;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndexBulkLoader.java
new file mode 100644
index 0000000..45a88a7
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndexBulkLoader.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.impls;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.IPageManager;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexMetadataFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
+import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+
+public abstract class AbstractTreeIndexBulkLoader extends PageWriteFailureCallback implements IIndexBulkLoader {
+    protected final IBufferCache bufferCache;
+    protected final IPageManager freePageManager;
+    protected final AbstractTreeIndex treeIndex;
+    protected final int fileId;
+    protected final MultiComparator cmp;
+    protected final int slotSize;
+    protected final int leafMaxBytes;
+    protected final int interiorMaxBytes;
+    protected final ArrayList<NodeFrontier> nodeFrontiers = new ArrayList<>();
+    protected final ITreeIndexMetadataFrame metaFrame;
+    protected final ITreeIndexTupleWriter tupleWriter;
+    protected ITreeIndexFrame leafFrame;
+    protected ITreeIndexFrame interiorFrame;
+    // Immutable bulk loaders write their root page at page -2, as needed e.g. by append-only file systems such as
+    // HDFS.  Since loading this tree relies on the root page actually being at that point, no further inserts into
+    // that tree are allowed.  Currently, this is not enforced.
+    protected boolean releasedLatches;
+    private final IFIFOPageWriter pageWriter;
+    protected List<ICachedPage> pagesToWrite;
+    private final ICompressedPageWriter compressedPageWriter;
+
+    protected AbstractTreeIndexBulkLoader(float fillFactor, IPageWriteCallback callback, ITreeIndex index)
+            throws HyracksDataException {
+        this(fillFactor, callback, index, index.getLeafFrameFactory().createFrame());
+    }
+
+    protected AbstractTreeIndexBulkLoader(float fillFactor, IPageWriteCallback callback, ITreeIndex index,
+            ITreeIndexFrame leafFrame) throws HyracksDataException {
+        this.bufferCache = index.getBufferCache();
+        this.freePageManager = index.getPageManager();
+        this.fileId = index.getFileId();
+        this.treeIndex = (AbstractTreeIndex) index;
+        this.leafFrame = leafFrame;
+        interiorFrame = treeIndex.getInteriorFrameFactory().createFrame();
+        metaFrame = freePageManager.createMetadataFrame();
+
+        pageWriter = bufferCache.createFIFOWriter(callback, this);
+
+        if (!treeIndex.isEmptyTree(leafFrame)) {
+            throw HyracksDataException.create(ErrorCode.CANNOT_BULK_LOAD_NON_EMPTY_TREE);
+        }
+
+        this.cmp = MultiComparator.create(treeIndex.getCmpFactories());
+
+        leafFrame.setMultiComparator(cmp);
+        interiorFrame.setMultiComparator(cmp);
+
+        tupleWriter = leafFrame.getTupleWriter();
+        NodeFrontier leafFrontier = new NodeFrontier(createTupleReference());
+        leafFrontier.pageId = freePageManager.takePage(metaFrame);
+        leafFrontier.page = bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId));
+
+        interiorFrame.setPage(leafFrontier.page);
+        interiorFrame.initBuffer((byte) 0);
+        interiorMaxBytes = (int) (interiorFrame.getBuffer().capacity() * fillFactor);
+
+        leafFrame.setPage(leafFrontier.page);
+        leafFrame.initBuffer((byte) 0);
+        leafMaxBytes = (int) (leafFrame.getBuffer().capacity() * fillFactor);
+        slotSize = leafFrame.getSlotSize();
+
+        nodeFrontiers.add(leafFrontier);
+        pagesToWrite = new ArrayList<>();
+        compressedPageWriter = bufferCache.getCompressedPageWriter(fileId);
+    }
+
+    protected ITreeIndexTupleReference createTupleReference() {
+        return leafFrame.createTupleReference();
+    }
+
+    protected void handleException() {
+        // Unlatch and unpin pages that weren't in the queue to avoid leaking memory.
+        compressedPageWriter.abort();
+        for (NodeFrontier nodeFrontier : nodeFrontiers) {
+            if (nodeFrontier != null && nodeFrontier.page != null) {
+                ICachedPage frontierPage = nodeFrontier.page;
+                if (frontierPage.confiscated()) {
+                    bufferCache.returnPage(frontierPage, false);
+                }
+            }
+        }
+        for (ICachedPage pageToDiscard : pagesToWrite) {
+            if (pageToDiscard != null) {
+                bufferCache.returnPage(pageToDiscard, false);
+            }
+        }
+        releasedLatches = true;
+    }
+
+    @Override
+    public void end() throws HyracksDataException {
+        if (hasFailed()) {
+            throw HyracksDataException.create(getFailure());
+        }
+        freePageManager.setRootPageId(treeIndex.getRootPageId());
+    }
+
+    protected void setRootPageId(int rootPage) {
+        treeIndex.rootPage = rootPage;
+    }
+
+    protected void addLevel() throws HyracksDataException {
+        NodeFrontier frontier = new NodeFrontier(tupleWriter.createTupleReference());
+        frontier.page = bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
+        frontier.pageId = -1;
+        frontier.lastTuple.setFieldCount(cmp.getKeyFieldCount());
+        interiorFrame.setPage(frontier.page);
+        interiorFrame.initBuffer((byte) nodeFrontiers.size());
+        nodeFrontiers.add(frontier);
+    }
+
+    public ITreeIndexFrame getLeafFrame() {
+        return leafFrame;
+    }
+
+    public void setLeafFrame(ITreeIndexFrame leafFrame) {
+        this.leafFrame = leafFrame;
+    }
+
+    public void write(ICachedPage cPage) throws HyracksDataException {
+        compressedPageWriter.prepareWrite(cPage);
+        pageWriter.write(cPage);
+    }
+
+    @Override
+    public void force() throws HyracksDataException {
+        bufferCache.force(fileId, false);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjector.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjector.java
new file mode 100644
index 0000000..00cb0c5
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjector.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.common.impls;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.common.projection.ITupleProjector;
+
+class DefaultTupleProjector implements ITupleProjector {
+    public static final ITupleProjector INSTANCE = new DefaultTupleProjector();
+
+    private DefaultTupleProjector() {
+    }
+
+    @Override
+    public void project(ITupleReference tuple, DataOutput dos, ArrayTupleBuilder tb) throws IOException {
+        for (int i = 0; i < tuple.getFieldCount(); i++) {
+            dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+            tb.addFieldEndOffset();
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjectorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjectorFactory.java
new file mode 100644
index 0000000..092982d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/DefaultTupleProjectorFactory.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.common.impls;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.projection.ITupleProjector;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
+
+public class DefaultTupleProjectorFactory implements ITupleProjectorFactory {
+    private static final long serialVersionUID = -4525893018744087821L;
+    public static final DefaultTupleProjectorFactory INSTANCE = new DefaultTupleProjectorFactory();
+
+    private DefaultTupleProjectorFactory() {
+    }
+
+    @Override
+    public ITupleProjector createTupleProjector(IHyracksTaskContext context) throws HyracksDataException {
+        return DefaultTupleProjector.INSTANCE;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/IndexAccessParameters.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/IndexAccessParameters.java
index 11d3cd5..63902be 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/IndexAccessParameters.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/IndexAccessParameters.java
@@ -59,6 +59,11 @@
         return paramMap;
     }
 
+    @Override
+    public <T> T getParameter(String key, Class<T> clazz) {
+        return paramMap != null && clazz.isInstance(paramMap.get(key)) ? clazz.cast(paramMap.get(key)) : null;
+    }
+
     public static IIndexAccessParameters createNoOpParams(IIndexCursorStats stats) {
         if (stats == NoOpIndexCursorStats.INSTANCE) {
             return NoOpIndexAccessParameters.INSTANCE;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpIndexAccessParameters.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpIndexAccessParameters.java
index fa7811c..76a1930 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpIndexAccessParameters.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpIndexAccessParameters.java
@@ -47,4 +47,9 @@
     public Map<String, Object> getParameters() {
         return paramMap;
     }
+
+    @Override
+    public <T> T getParameter(String key, Class<T> clazz) {
+        return null;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/pom.xml
new file mode 100644
index 0000000..9e4a297
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/pom.xml
@@ -0,0 +1,91 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT 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>
+  <artifactId>hyracks-storage-am-lsm-btree-column</artifactId>
+  <parent>
+    <groupId>org.apache.hyracks</groupId>
+    <artifactId>hyracks</artifactId>
+    <version>0.3.8-SNAPSHOT</version>
+  </parent>
+  <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>
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-storage-am-btree</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-storage-am-lsm-btree</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-storage-am-bloomfilter</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-storage-am-lsm-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-dataflow-common</artifactId>
+      <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.hyracks</groupId>
+      <artifactId>hyracks-api</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-storage-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-data-std</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-util</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+  </dependencies>
+</project>
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleReader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleReader.java
new file mode 100644
index 0000000..7db792b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleReader.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.hyracks.storage.am.lsm.btree.column.api;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.AbstractColumnBTreeLeafFrame;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+
+/**
+ * Provided for columnar read tuple reference
+ */
+public abstract class AbstractColumnTupleReader extends AbstractTupleWriterDisabledMethods {
+    public abstract IColumnTupleIterator createTupleIterator(ColumnBTreeReadLeafFrame frame, int componentIndex,
+            IColumnReadMultiPageOp multiPageOp);
+
+    /**
+     * Currently fixed to 4-byte per offset
+     *
+     * @param buf         buffer of Page0
+     * @param columnIndex column index
+     * @return column offset
+     * @see AbstractColumnTupleWriter#getColumnOffsetsSize()
+     */
+    public final int getColumnOffset(ByteBuffer buf, int columnIndex) {
+        return buf.getInt(AbstractColumnBTreeLeafFrame.HEADER_SIZE + columnIndex * Integer.BYTES);
+    }
+
+    @Override
+    public final int bytesRequired(ITupleReference tuple) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleReaderWriterFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleReaderWriterFactory.java
new file mode 100644
index 0000000..774bbb9
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleReaderWriterFactory.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.hyracks.storage.am.lsm.btree.column.api;
+
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+
+/**
+ * For columns, there are two types for {@link ITreeIndexTupleWriter} one used during write and another during read
+ */
+public abstract class AbstractColumnTupleReaderWriterFactory implements ITreeIndexTupleWriterFactory {
+    private static final long serialVersionUID = -2377235465942457248L;
+    protected final int pageSize;
+    protected final int maxNumberOfTuples;
+    protected final float tolerance;
+
+    /**
+     * Tuple reader/writer factory
+     *
+     * @param pageSize          {@link IBufferCache} page size
+     * @param maxNumberOfTuples maximum number of tuples stored per a mega leaf page
+     * @param tolerance         percentage of tolerated empty space
+     */
+    protected AbstractColumnTupleReaderWriterFactory(int pageSize, int maxNumberOfTuples, float tolerance) {
+        this.pageSize = pageSize;
+        this.maxNumberOfTuples = maxNumberOfTuples;
+        this.tolerance = tolerance;
+    }
+
+    /**
+     * Create columnar tuple writer
+     *
+     * @param columnMetadata writer column metadata
+     */
+    public abstract AbstractColumnTupleWriter createColumnWriter(IColumnMetadata columnMetadata);
+
+    /**
+     * Create columnar tuple reader
+     *
+     * @param columnProjectionInfo column projection info for either query or merge
+     */
+    public abstract AbstractColumnTupleReader createColumnReader(IColumnProjectionInfo columnProjectionInfo);
+
+    @Override
+    public final ITreeIndexTupleWriter createTupleWriter() {
+        throw new UnsupportedOperationException("Operation is not supported for " + getClass().getName());
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleWriter.java
new file mode 100644
index 0000000..0c19ce7
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractColumnTupleWriter.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.btree.column.api;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * Columnar Tuple Writer:
+ * <p>
+ * The writer does not write directly to the page(s) buffer but write to internal temporary buffers (provided by
+ * {@link IColumnWriteMultiPageOp} until the header page (or page0) is full or the number of tuples equals to the
+ * {@link #getMaxNumberOfTuples()}
+ * Then, the columns are flushed to disk.
+ * <p>
+ * Contract:
+ * - Initially, the writer has to set multiPageOp by calling {@link #init(IColumnWriteMultiPageOp)}
+ * - For each write, the caller should check if adding a tuple does not exceed the {@link #getMaxNumberOfTuples()} or
+ * the on-disk page size (called stopping condition)
+ * - If the stopping condition is reached, then {@link #flush(ByteBuffer)} needed to be called
+ * <p>
+ * Hyracks visibility:
+ * - Columns are written as blobs (i.e., not interpretable by Hyracks)
+ * - Hyracks only aware of where each column at
+ */
+public abstract class AbstractColumnTupleWriter extends AbstractTupleWriterDisabledMethods {
+    /**
+     * Set the writer with {@link IColumnWriteMultiPageOp} to allocate columns for their writers
+     *
+     * @param multiPageOp multiPageOp
+     */
+    public abstract void init(IColumnWriteMultiPageOp multiPageOp) throws HyracksDataException;
+
+    /**
+     * @return The current number of columns
+     */
+    public abstract int getNumberOfColumns();
+
+    /**
+     * Currently, a column offset takes 4-byte (fixed). But in the future, we can reformat the offsets. For example,
+     * we can store index-offset pairs if we encounter a sparse columns (i.e., most columns are just nulls).
+     *
+     * @return the size needed to store columns' offsets
+     */
+    public final int getColumnOffsetsSize() {
+        return Integer.BYTES * getNumberOfColumns();
+    }
+
+    /**
+     * @return maximum number of tuples to be stored per page (i.e., page0)
+     */
+    public abstract int getMaxNumberOfTuples();
+
+    /**
+     * @return page0 occupied space
+     */
+    public abstract int getOccupiedSpace();
+
+    /**
+     * Writes the tuple into a temporary internal buffers
+     *
+     * @param tuple The tuple to be written
+     */
+    public abstract void writeTuple(ITupleReference tuple) throws HyracksDataException;
+
+    /**
+     * Flush all columns from the internal buffers to the page buffer
+     *
+     * @return the allocated space used to write tuples
+     */
+    public abstract int flush(ByteBuffer pageZero) throws HyracksDataException;
+
+    /**
+     * Close the current writer and release all allocated temporary buffers
+     */
+    public abstract void close();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractTupleWriterDisabledMethods.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractTupleWriterDisabledMethods.java
new file mode 100644
index 0000000..abc5ef0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/AbstractTupleWriterDisabledMethods.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.hyracks.storage.am.lsm.btree.column.api;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+
+/**
+ * Disable all row write methods
+ */
+public abstract class AbstractTupleWriterDisabledMethods implements ITreeIndexTupleWriter {
+    protected static final String UNSUPPORTED_OPERATION_MSG = "Operation is not supported for columnar tuple reader";
+
+    /* ***********************************************
+     * Disable write-related operations
+     * ***********************************************
+     */
+
+    @Override
+    public final ITreeIndexTupleReference createTupleReference() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int writeTuple(ITupleReference tuple, ByteBuffer targetBuf, int targetOff) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int writeTuple(ITupleReference tuple, byte[] targetBuf, int targetOff) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int writeTupleFields(ITupleReference tuple, int startField, int numFields, byte[] targetBuf,
+            int targetOff) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int bytesRequired(ITupleReference tuple, int startField, int numFields) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int getCopySpaceRequired(ITupleReference tuple) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void setUpdated(boolean isUpdated) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnBufferProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnBufferProvider.java
new file mode 100644
index 0000000..d0b5e12
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnBufferProvider.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.hyracks.storage.am.lsm.btree.column.api;
+
+import java.nio.ByteBuffer;
+import java.util.Queue;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+
+/**
+ * A proxy to call {@link IBufferCache} operations. Each column should have its own buffer provider
+ */
+public interface IColumnBufferProvider {
+    /**
+     * Calling this method would pin all the pages of the requested columns from the buffer cache
+     *
+     * @param frame the frame for Page0
+     */
+    void reset(ColumnBTreeReadLeafFrame frame) throws HyracksDataException;
+
+    /**
+     * Return all the pages for a column
+     *
+     * @param buffers queue for all pages of a column
+     */
+    void readAll(Queue<ByteBuffer> buffers) throws HyracksDataException;
+
+    /**
+     * Release all the column pages (i.e., unpin all column pages)
+     */
+    void releaseAll() throws HyracksDataException;
+
+    /**
+     * @return a buffer of a column (in case there is only a single page for a column)
+     */
+    ByteBuffer getBuffer();
+
+    /**
+     * @return the actual length (in bytes) for all the column's pages
+     */
+    int getLength();
+
+    /**
+     * @return the column index
+     */
+    int getColumnIndex();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManager.java
new file mode 100644
index 0000000..278ea03
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManager.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.hyracks.storage.am.lsm.btree.column.api;
+
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnTupleProjector;
+
+public interface IColumnManager {
+    /**
+     * Activate the columnar manager for an empty dataset
+     *
+     * @return empty column metadata
+     */
+    IColumnMetadata activate() throws HyracksDataException;
+
+    /**
+     * Activate the column manager for a non-empty dataset
+     *
+     * @param metadata column metadata value from the latest component metadata
+     * @return latest column metadata
+     */
+    IColumnMetadata activate(IValueReference metadata) throws HyracksDataException;
+
+    /**
+     * Create merge column metadata for a newly created merge component
+     *
+     * @param metadata         latest column metadata value stored in the metadata page
+     * @param componentsTuples tuples of the merging components
+     * @return column metadata for a new merged component
+     */
+    IColumnMetadata createMergeColumnMetadata(IValueReference metadata, List<IColumnTupleIterator> componentsTuples)
+            throws HyracksDataException;
+
+    /**
+     * Create tuple projector for reading the merging components. The merge tuple projector will return all columns
+     *
+     * @return merge tuple projector
+     */
+    IColumnTupleProjector getMergeColumnProjector();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManagerFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManagerFactory.java
new file mode 100644
index 0000000..a2dfbcf
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnManagerFactory.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.hyracks.storage.am.lsm.btree.column.api;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+
+public interface IColumnManagerFactory extends Serializable, IJsonSerializable {
+    /**
+     * @return a new instance of {@link IColumnManager}
+     */
+    IColumnManager createColumnManager();
+
+    /**
+     * Get column tuple reader/writer for the {@link LSMIOOperationType#LOAD}
+     */
+    AbstractColumnTupleReaderWriterFactory getLoadColumnTupleReaderWriterFactory();
+
+    /**
+     * Get column tuple reader/writer for the {@link LSMIOOperationType#FLUSH}
+     */
+    AbstractColumnTupleReaderWriterFactory getFlushColumnTupleReaderWriterFactory();
+
+    /**
+     * Get column tuple reader/writer for the {@link LSMIOOperationType#MERGE}
+     */
+    AbstractColumnTupleReaderWriterFactory createMergeColumnTupleReaderWriterFactory();
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnMetadata.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnMetadata.java
new file mode 100644
index 0000000..4c23b97
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnMetadata.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.hyracks.storage.am.lsm.btree.column.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.util.annotations.NotThreadSafe;
+
+/**
+ * A holder for the columnar metadata.
+ * Modifications on the columnar metadata are not thread safe.
+ */
+@NotThreadSafe
+public interface IColumnMetadata {
+    /**
+     * @return a serialized version of the columns metadata
+     */
+    IValueReference serializeColumnsMetadata() throws HyracksDataException;
+
+    /**
+     * abort in case of an error. This should clean up any artifact
+     */
+    void abort() throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnReadMultiPageOp.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnReadMultiPageOp.java
new file mode 100644
index 0000000..f43a6e9
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnReadMultiPageOp.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.column.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+
+/**
+ * A proxy to call {@link IBufferCache} read columns' pages
+ * Implementer should be aware to unpin all pages in case of an error
+ */
+public interface IColumnReadMultiPageOp {
+    /**
+     * Pin a column page
+     *
+     * @return a page that belongs to a column
+     */
+    ICachedPage pin(int pageId) throws HyracksDataException;
+
+    /**
+     * Unpin a pinned column page
+     */
+    void unpin(ICachedPage page) throws HyracksDataException;
+
+    /**
+     * Return {@link IBufferCache} page size
+     *
+     * @see IBufferCache#getPageSize()
+     */
+    int getPageSize();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnTupleIterator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnTupleIterator.java
new file mode 100644
index 0000000..2ffa1bb
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnTupleIterator.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.hyracks.storage.am.lsm.btree.column.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+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.ILSMTreeTupleReference;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+
+/**
+ * A tuple representation that combines all the columns. It simply provides a way to iterate over tuples for a given
+ * set that could span multiple pages.
+ */
+public interface IColumnTupleIterator extends ILSMTreeTupleReference, Comparable<IColumnTupleIterator> {
+    /**
+     * Reset the iterator starting at the provided index
+     *
+     * @param startIndex start from the tuple at this index
+     */
+    void reset(int startIndex) throws HyracksDataException;
+
+    /**
+     * Mark {@link IColumnTupleIterator} as consumed
+     */
+    void consume();
+
+    /**
+     * @return true if the {@link IColumnTupleIterator} is consumed, false otherwise
+     */
+    boolean isConsumed();
+
+    /**
+     * Skip a number of tuples
+     *
+     * @param count the number of tuples that needed to be skipped
+     */
+    void skip(int count) throws HyracksDataException;
+
+    /**
+     * Move to the next tuple
+     */
+    void next() throws HyracksDataException;
+
+    /**
+     * Notifies that the last tuple has been consumed
+     */
+    void lastTupleReached() throws HyracksDataException;
+
+    /**
+     * The component index is the same as the index of a component in an {@link ILSMIndexCursor}
+     *
+     * @return From which {@link ILSMComponent} this iterator is for
+     */
+    int getComponentIndex();
+
+    /**
+     * Calls {@link IBufferCache#unpin(ICachedPage)} for all columns' pages
+     */
+    void unpinColumnsPages() throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnWriteMultiPageOp.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnWriteMultiPageOp.java
new file mode 100644
index 0000000..2309fe1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/IColumnWriteMultiPageOp.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.hyracks.storage.am.lsm.btree.column.api;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+
+/**
+ * A proxy to call {@link IBufferCache} writing methods
+ * <p>
+ * An instance of this interface is responsible for returning all confiscated pages back to {@link IBufferCache} upon
+ * failures. Temporary buffers should be returned to the {@link IBufferCache} once the multi-page operation is finished.
+ * <p>
+ * Users of an instance of this interface should not expect the temporary buffers will last after the multi-page
+ * operation is finished.
+ */
+public interface IColumnWriteMultiPageOp {
+    /**
+     * @return a buffer that correspond to a page in a file
+     */
+    ByteBuffer confiscatePersistent() throws HyracksDataException;
+
+    /**
+     * Persist all confiscated persistent buffers to disk
+     */
+    void persist() throws HyracksDataException;
+
+    /**
+     * @return the number confiscated persistent pages
+     */
+    int getNumberOfPersistentBuffers();
+
+    /**
+     * @return a {@link IBufferCache}-backed buffer for temporary use
+     */
+    ByteBuffer confiscateTemporary() throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnProjectionInfo.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnProjectionInfo.java
new file mode 100644
index 0000000..1506433
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnProjectionInfo.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.hyracks.storage.am.lsm.btree.column.api.projection;
+
+/**
+ * Gets information about the requested columns
+ */
+public interface IColumnProjectionInfo {
+    /**
+     * @param ordinal position of the requested column
+     * @return column index given the ordinal number of the requested column
+     */
+    int getColumnIndex(int ordinal);
+
+    /**
+     * @return total number of requested columns
+     */
+    int getNumberOfProjectedColumns();
+
+    /**
+     * @return number of primary keys
+     */
+    int getNumberOfPrimaryKeys();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnTupleProjector.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnTupleProjector.java
new file mode 100644
index 0000000..c1301da
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/api/projection/IColumnTupleProjector.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.api.projection;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.lsm.common.api.IComponentMetadata;
+import org.apache.hyracks.storage.common.projection.ITupleProjector;
+
+/**
+ * A specialized {@link ITupleProjector} for columnar datasets
+ */
+public interface IColumnTupleProjector extends ITupleProjector {
+    /**
+     * Create projection information
+     *
+     * @param columnMetadata the latest component's raw column metadata as stored in {@link IComponentMetadata}
+     * @return projection information
+     */
+    IColumnProjectionInfo createProjectionInfo(IValueReference columnMetadata) throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/AbstractColumnBTreeLeafFrame.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/AbstractColumnBTreeLeafFrame.java
new file mode 100644
index 0000000..9aeafa4
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/AbstractColumnBTreeLeafFrame.java
@@ -0,0 +1,267 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.impls.btree;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.ISlotManager;
+import org.apache.hyracks.storage.am.common.api.ISplitKey;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
+import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IExtraPageBlockHelper;
+
+/**
+ * Disable all unsupported/unused operations
+ */
+public abstract class AbstractColumnBTreeLeafFrame implements ITreeIndexFrame {
+    private static final String UNSUPPORTED_OPERATION_MSG = "Operation is not supported";
+
+    /*
+     * Remap the BTreeNSMFrame pointers for columnar pages
+     */
+    //Same as before
+    public static final int TUPLE_COUNT_OFFSET = Constants.TUPLE_COUNT_OFFSET;
+    //Previously Renaming
+    public static final int NUMBER_OF_COLUMNS_OFFSET = Constants.FREE_SPACE_OFFSET;
+    //Previously first four byte of LSN.
+    public static final int LEFT_MOST_KEY_OFFSET = Constants.RESERVED_HEADER_SIZE;
+    //Previously last four byte of LSN.
+    public static final int RIGHT_MOST_KEY_OFFSET = LEFT_MOST_KEY_OFFSET + 4;
+    /**
+     * Currently, a column offset takes 4-byte (fixed). But in the future, we can reformat the offsets. For example,
+     * we can store index-offset pairs if we encounter a sparse columns (i.e., most columns are just nulls). This
+     * reformatting could be indicated by the FLAG byte.
+     *
+     * @see AbstractColumnTupleWriter#getColumnOffsetsSize()
+     */
+    public static final int SIZE_OF_COLUMNS_OFFSETS_OFFSET = RIGHT_MOST_KEY_OFFSET + 4;
+    //Total number of columns pages
+    public static final int NUMBER_OF_COLUMN_PAGES = SIZE_OF_COLUMNS_OFFSETS_OFFSET + 4;
+    //A flag (used in NSM to indicate small and large pages). We can reuse it as explained above
+    public static final int FLAG_OFFSET = NUMBER_OF_COLUMN_PAGES + 4;
+    public static final int NEXT_LEAF_OFFSET = FLAG_OFFSET + 1;
+    public static final int HEADER_SIZE = NEXT_LEAF_OFFSET + 4;
+
+    protected final ITreeIndexTupleWriter rowTupleWriter;
+
+    protected MultiComparator cmp;
+    protected ICachedPage page;
+    protected ByteBuffer buf;
+
+    AbstractColumnBTreeLeafFrame(ITreeIndexTupleWriter rowTupleWriter) {
+        this.rowTupleWriter = rowTupleWriter;
+    }
+
+    /* ****************************************************************************
+     * Needed by both read and write
+     * ****************************************************************************
+     */
+
+    @Override
+    public final ITreeIndexTupleWriter getTupleWriter() {
+        return rowTupleWriter;
+    }
+
+    @Override
+    public final void setMultiComparator(MultiComparator cmp) {
+        this.cmp = cmp;
+    }
+
+    @Override
+    public final void setPage(ICachedPage page) {
+        this.page = page;
+        this.buf = page.getBuffer();
+        buf.clear();
+        buf.position(HEADER_SIZE);
+    }
+
+    @Override
+    public final ICachedPage getPage() {
+        return page;
+    }
+
+    @Override
+    public final ByteBuffer getBuffer() {
+        return buf;
+    }
+
+    @Override
+    public final boolean isLeaf() {
+        return true;
+    }
+
+    @Override
+    public final boolean isInterior() {
+        return false;
+    }
+
+    @Override
+    public final int getPageHeaderSize() {
+        return HEADER_SIZE;
+    }
+
+    /* ****************************************************************************
+     * Operations that are needed by either read or write
+     * ****************************************************************************
+     */
+
+    @Override
+    public void initBuffer(byte level) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public int getTupleCount() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public ITreeIndexTupleReference createTupleReference() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public ITupleReference getLeftmostTuple() throws HyracksDataException {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public ITupleReference getRightmostTuple() throws HyracksDataException {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    /* ****************************************************************************
+     * Unsupported Operations
+     * ****************************************************************************
+     */
+
+    @Override
+    public final String printHeader() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final byte getLevel() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void setLevel(byte level) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int getBytesRequiredToWriteTuple(ITupleReference tuple) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final FrameOpSpaceStatus hasSpaceInsert(ITupleReference tuple) throws HyracksDataException {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void insert(ITupleReference tuple, int tupleIndex) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final FrameOpSpaceStatus hasSpaceUpdate(ITupleReference newTuple, int oldTupleIndex) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void update(ITupleReference newTuple, int oldTupleIndex, boolean inPlace) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void delete(ITupleReference tuple, int tupleIndex) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final boolean compact() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final boolean compress() throws HyracksDataException {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int getTupleOffset(int slotNum) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int getTotalFreeSpace() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void setPageLsn(long pageLsn) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final long getPageLsn() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int getMaxTupleSize(int pageSize) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey splitKey,
+            IExtraPageBlockHelper extraPageBlockHelper, IBufferCache bufferCache) throws HyracksDataException {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final ISlotManager getSlotManager() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final int getSlotSize() {
+        return 0;
+    }
+
+    @Override
+    public final int getFreeSpaceOff() {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void setFreeSpaceOff(int freeSpace) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTree.java
new file mode 100644
index 0000000..fcee22c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTree.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.impls.btree;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.util.HyracksConstants;
+import org.apache.hyracks.storage.am.btree.impls.DiskBTree;
+import org.apache.hyracks.storage.am.common.api.IPageManager;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnMetadata;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+import org.apache.hyracks.storage.common.IIndexAccessParameters;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.IIndexCursorStats;
+import org.apache.hyracks.storage.common.NoOpIndexCursorStats;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+
+public class ColumnBTree extends DiskBTree {
+    public ColumnBTree(IBufferCache bufferCache, IPageManager freePageManager,
+            ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
+            IBinaryComparatorFactory[] cmpFactories, int fieldCount, FileReference file) {
+        super(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories, fieldCount, file);
+    }
+
+    @Override
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+            boolean checkIfEmptyIndex, IPageWriteCallback callback) {
+        throw new IllegalAccessError("Missing write column metadata");
+    }
+
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback,
+            IColumnMetadata columnMetadata) throws HyracksDataException {
+        ColumnBTreeLeafFrameFactory columnLeafFrameFactory = (ColumnBTreeLeafFrameFactory) leafFrameFactory;
+        ColumnBTreeWriteLeafFrame writeLeafFrame = columnLeafFrameFactory.createWriterFrame(columnMetadata);
+        return new ColumnBTreeBulkloader(fillFactor, verifyInput, callback, this, writeLeafFrame);
+    }
+
+    @Override
+    public BTreeAccessor createAccessor(IIndexAccessParameters iap) {
+        throw new IllegalArgumentException("Use createAccessor(IIndexAccessParameters, int, IColumnTupleProjector)");
+    }
+
+    public BTreeAccessor createAccessor(IIndexAccessParameters iap, int index, IColumnProjectionInfo projectionInfo) {
+        return new ColumnBTreeAccessor(this, iap, index, projectionInfo);
+    }
+
+    public class ColumnBTreeAccessor extends DiskBTreeAccessor {
+        private final int index;
+        private final IColumnProjectionInfo projectionInfo;
+
+        public ColumnBTreeAccessor(ColumnBTree btree, IIndexAccessParameters iap, int index,
+                IColumnProjectionInfo projectionInfo) {
+            super(btree, iap);
+            this.index = index;
+            this.projectionInfo = projectionInfo;
+        }
+
+        @Override
+        public ITreeIndexCursor createSearchCursor(boolean exclusive) {
+            ColumnBTreeLeafFrameFactory columnLeafFrameFactory = (ColumnBTreeLeafFrameFactory) leafFrameFactory;
+            ColumnBTreeReadLeafFrame readLeafFrame = columnLeafFrameFactory.createReadFrame(projectionInfo);
+            return new ColumnBTreeRangeSearchCursor(readLeafFrame, (IIndexCursorStats) iap.getParameters()
+                    .getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE), index);
+        }
+
+        @Override
+        public ITreeIndexCursor createPointCursor(boolean exclusive, boolean stateful) {
+            ColumnBTreeLeafFrameFactory columnLeafFrameFactory = (ColumnBTreeLeafFrameFactory) leafFrameFactory;
+            ColumnBTreeReadLeafFrame readLeafFrame = columnLeafFrameFactory.createReadFrame(projectionInfo);
+            return new ColumnBTreePointSearchCursor(readLeafFrame, (IIndexCursorStats) iap.getParameters()
+                    .getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE), index);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeBulkloader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeBulkloader.java
new file mode 100644
index 0000000..48bd180
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeBulkloader.java
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.impls.btree;
+
+import java.nio.ByteBuffer;
+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.btree.impls.BTreeNSMBulkLoader;
+import org.apache.hyracks.storage.am.btree.impls.BTreeSplitKey;
+import org.apache.hyracks.storage.am.common.api.ISplitKey;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+import org.apache.hyracks.storage.common.buffercache.CachedPage;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+
+public final class ColumnBTreeBulkloader extends BTreeNSMBulkLoader implements IColumnWriteMultiPageOp {
+    private final List<CachedPage> columnsPages;
+    private final List<CachedPage> tempConfiscatedPages;
+    private final ColumnBTreeWriteLeafFrame columnarFrame;
+    private final AbstractColumnTupleWriter columnWriter;
+    private final ISplitKey lowKey;
+    private boolean setLowKey;
+    private int tupleCount;
+
+    public ColumnBTreeBulkloader(float fillFactor, boolean verifyInput, IPageWriteCallback callback, ITreeIndex index,
+            ITreeIndexFrame leafFrame) throws HyracksDataException {
+        super(fillFactor, verifyInput, callback, index, leafFrame);
+        columnsPages = new ArrayList<>();
+        tempConfiscatedPages = new ArrayList<>();
+        columnarFrame = (ColumnBTreeWriteLeafFrame) leafFrame;
+        columnWriter = columnarFrame.getColumnTupleWriter();
+        columnWriter.init(this);
+        lowKey = new BTreeSplitKey(tupleWriter.createTupleReference());
+        lowKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
+        setLowKey = true;
+    }
+
+    @Override
+    public void add(ITupleReference tuple) throws HyracksDataException {
+        if (isFull(tuple)) {
+            writeFullLeafPage();
+            confiscateNewLeafPage();
+        }
+        //Save the key of the last inserted tuple
+        setMinMaxKeys(tuple);
+        columnWriter.writeTuple(tuple);
+        tupleCount++;
+    }
+
+    @Override
+    protected ITreeIndexTupleReference createTupleReference() {
+        return tupleWriter.createTupleReference();
+    }
+
+    private boolean isFull(ITupleReference tuple) {
+        if (tupleCount == 0) {
+            return false;
+        } else if (tupleCount >= columnWriter.getMaxNumberOfTuples()) {
+            //We reached the maximum number of tuples
+            return true;
+        }
+        int requiredFreeSpace = AbstractColumnBTreeLeafFrame.HEADER_SIZE;
+        //Columns' Offsets
+        requiredFreeSpace += columnWriter.getColumnOffsetsSize();
+        //Occupied space from previous writes
+        requiredFreeSpace += columnWriter.getOccupiedSpace();
+        //min and max tuples' sizes
+        requiredFreeSpace += lowKey.getTuple().getTupleSize() + splitKey.getTuple().getTupleSize();
+        //New tuple required space
+        requiredFreeSpace += columnWriter.bytesRequired(tuple);
+        return bufferCache.getPageSize() <= requiredFreeSpace;
+    }
+
+    private void setMinMaxKeys(ITupleReference tuple) {
+        //Set max key
+        setSplitKey(splitKey, tuple);
+        if (setLowKey) {
+            setSplitKey(lowKey, tuple);
+            lowKey.getTuple().resetByTupleOffset(lowKey.getBuffer().array(), 0);
+            setLowKey = false;
+        }
+    }
+
+    @Override
+    public void end() throws HyracksDataException {
+        if (tupleCount > 0) {
+            splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+            columnarFrame.flush(columnWriter, tupleCount, this, lowKey.getTuple(), splitKey.getTuple());
+        }
+        columnWriter.close();
+        //We are done, return any temporary confiscated pages
+        for (ICachedPage page : tempConfiscatedPages) {
+            bufferCache.returnPage(page, false);
+        }
+        tempConfiscatedPages.clear();
+        //Where Page0 and columns pages will be written
+        super.end();
+    }
+
+    @Override
+    protected void writeFullLeafPage() throws HyracksDataException {
+        NodeFrontier leafFrontier = nodeFrontiers.get(0);
+        splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+        splitKey.setLeftPage(leafFrontier.pageId);
+        if (tupleCount > 0) {
+            //We need to flush columns to confiscate all columns pages first before calling propagateBulk
+            columnarFrame.flush(columnWriter, tupleCount, this, lowKey.getTuple(), splitKey.getTuple());
+        }
+
+        propagateBulk(1, pagesToWrite);
+
+        //Take a page for the next leaf
+        leafFrontier.pageId = freePageManager.takePage(metaFrame);
+        columnarFrame.setNextLeaf(leafFrontier.pageId);
+
+        /*
+         * Write columns' pages first to ensure they (columns' pages) are written before pageZero.
+         * It ensures pageZero does not land in between columns' pages if compression is enabled
+         */
+        writeColumnsPages();
+        //Then write page0
+        write(leafFrontier.page);
+
+        //Write interior nodes after writing columns pages
+        for (ICachedPage c : pagesToWrite) {
+            write(c);
+        }
+
+        pagesToWrite.clear();
+        splitKey.setRightPage(leafFrontier.pageId);
+        setLowKey = true;
+        tupleCount = 0;
+    }
+
+    @Override
+    protected void writeLastLeaf(ICachedPage page) throws HyracksDataException {
+        /*
+         * Write columns' pages first to ensure they (columns' pages) are written before pageZero.
+         * It ensures pageZero does not land in between columns' pages if compression is enabled
+         */
+        writeColumnsPages();
+        super.writeLastLeaf(page);
+    }
+
+    private void writeColumnsPages() throws HyracksDataException {
+        for (ICachedPage c : columnsPages) {
+            write(c);
+        }
+        columnsPages.clear();
+    }
+
+    @Override
+    public void abort() throws HyracksDataException {
+        for (ICachedPage page : columnsPages) {
+            bufferCache.returnPage(page, false);
+        }
+
+        for (ICachedPage page : tempConfiscatedPages) {
+            bufferCache.returnPage(page, false);
+        }
+        super.abort();
+    }
+
+    private void setSplitKey(ISplitKey splitKey, ITupleReference tuple) {
+        int splitKeySize = tupleWriter.bytesRequired(tuple, 0, cmp.getKeyFieldCount());
+        splitKey.initData(splitKeySize);
+        tupleWriter.writeTupleFields(tuple, 0, cmp.getKeyFieldCount(), splitKey.getBuffer().array(), 0);
+    }
+
+    /*
+     * ***********************************************************
+     * IColumnWriteMultiPageOp
+     * ***********************************************************
+     */
+
+    @Override
+    public ByteBuffer confiscatePersistent() throws HyracksDataException {
+        int pageId = freePageManager.takePage(metaFrame);
+        long dpid = BufferedFileHandle.getDiskPageId(fileId, pageId);
+        CachedPage page = (CachedPage) bufferCache.confiscatePage(dpid);
+        columnsPages.add(page);
+        return page.getBuffer();
+    }
+
+    @Override
+    public void persist() throws HyracksDataException {
+        writeColumnsPages();
+    }
+
+    @Override
+    public int getNumberOfPersistentBuffers() {
+        return columnsPages.size();
+    }
+
+    @Override
+    public ByteBuffer confiscateTemporary() throws HyracksDataException {
+        CachedPage page = (CachedPage) bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
+        tempConfiscatedPages.add(page);
+        return page.getBuffer();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeFactory.java
new file mode 100644
index 0000000..1b9e198
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeFactory.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.impls.btree;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+
+public class ColumnBTreeFactory extends TreeIndexFactory<ColumnBTree> {
+    public ColumnBTreeFactory(IIOManager ioManager, IBufferCache bufferCache,
+            IPageManagerFactory freePageManagerFactory, ITreeIndexFrameFactory interiorFrameFactory,
+            ITreeIndexFrameFactory leafFrameFactory, IBinaryComparatorFactory[] cmpFactories, int fieldCount) {
+        super(ioManager, bufferCache, freePageManagerFactory, interiorFrameFactory, leafFrameFactory, cmpFactories,
+                fieldCount);
+    }
+
+    @Override
+    public ColumnBTree createIndexInstance(FileReference file) throws HyracksDataException {
+        return new ColumnBTree(bufferCache, freePageManagerFactory.createPageManager(bufferCache), interiorFrameFactory,
+                leafFrameFactory, cmpFactories, fieldCount, file);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeLeafFrameFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeLeafFrameFactory.java
new file mode 100644
index 0000000..31d85bd
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeLeafFrameFactory.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.hyracks.storage.am.lsm.btree.column.impls.btree;
+
+import org.apache.hyracks.storage.am.btree.frames.BTreeNSMLeafFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReader;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReaderWriterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnMetadata;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+
+public class ColumnBTreeLeafFrameFactory implements ITreeIndexFrameFactory {
+    private static final long serialVersionUID = 4136035898137820322L;
+    private final ITreeIndexTupleWriterFactory rowTupleWriterFactory;
+    private final AbstractColumnTupleReaderWriterFactory columnTupleWriterFactory;
+
+    public ColumnBTreeLeafFrameFactory(ITreeIndexTupleWriterFactory rowTupleWriterFactory,
+            AbstractColumnTupleReaderWriterFactory columnTupleWriterFactory) {
+        this.rowTupleWriterFactory = rowTupleWriterFactory;
+        this.columnTupleWriterFactory = columnTupleWriterFactory;
+    }
+
+    @Override
+    public ITreeIndexFrame createFrame() {
+        //Create a dummy leaf frame
+        return new BTreeNSMLeafFrame(rowTupleWriterFactory.createTupleWriter());
+    }
+
+    @Override
+    public ITreeIndexTupleWriterFactory getTupleWriterFactory() {
+        return rowTupleWriterFactory;
+    }
+
+    public ColumnBTreeWriteLeafFrame createWriterFrame(IColumnMetadata columnMetadata) {
+        ITreeIndexTupleWriter rowTupleWriter = rowTupleWriterFactory.createTupleWriter();
+        AbstractColumnTupleWriter columnTupleWriter = columnTupleWriterFactory.createColumnWriter(columnMetadata);
+        return new ColumnBTreeWriteLeafFrame(rowTupleWriter, columnTupleWriter);
+    }
+
+    public ColumnBTreeReadLeafFrame createReadFrame(IColumnProjectionInfo columnProjectionInfo) {
+        ITreeIndexTupleWriter rowTupleWriter = rowTupleWriterFactory.createTupleWriter();
+        AbstractColumnTupleReader columnTupleReader = columnTupleWriterFactory.createColumnReader(columnProjectionInfo);
+        return new ColumnBTreeReadLeafFrame(rowTupleWriter, columnTupleReader);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreePointSearchCursor.java
new file mode 100644
index 0000000..c93e77e
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreePointSearchCursor.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.hyracks.storage.am.lsm.btree.column.impls.btree;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.api.IDiskBTreeStatefulPointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.common.IIndexCursorStats;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+
+public class ColumnBTreePointSearchCursor extends ColumnBTreeRangeSearchCursor
+        implements IDiskBTreeStatefulPointSearchCursor {
+
+    public ColumnBTreePointSearchCursor(ColumnBTreeReadLeafFrame frame, IIndexCursorStats stats, int index) {
+        super(frame, stats, index);
+    }
+
+    @Override
+    public void doClose() throws HyracksDataException {
+        pageId = IBufferCache.INVALID_PAGEID;
+        super.doClose();
+    }
+
+    @Override
+    public int getLastPageId() {
+        return pageId;
+    }
+
+    @Override
+    public void setCursorToNextKey(ISearchPredicate searchPred) throws HyracksDataException {
+        initCursorPosition(searchPred);
+    }
+
+    @Override
+    public ITreeIndexFrame getFrame() {
+        return frame;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeRangeSearchCursor.java
new file mode 100644
index 0000000..d0b7e2b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeRangeSearchCursor.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.hyracks.storage.am.lsm.btree.column.impls.btree;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.impls.BTreeCursorInitialState;
+import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnReadMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
+import org.apache.hyracks.storage.common.ICursorInitialState;
+import org.apache.hyracks.storage.common.IIndexCursorStats;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+
+public class ColumnBTreeRangeSearchCursor extends EnforcedIndexCursor
+        implements ITreeIndexCursor, IColumnReadMultiPageOp {
+
+    protected final ColumnBTreeReadLeafFrame frame;
+    protected final IColumnTupleIterator frameTuple;
+
+    protected IBufferCache bufferCache = null;
+    protected int fileId;
+
+    protected int pageId;
+    protected ICachedPage page0 = null;
+
+    protected final RangePredicate reusablePredicate;
+    protected MultiComparator originalKeyCmp;
+
+    protected RangePredicate pred;
+    protected ITupleReference lowKey;
+    protected ITupleReference highKey;
+    protected boolean firstNextCall;
+
+    protected final IIndexCursorStats stats;
+
+    public ColumnBTreeRangeSearchCursor(ColumnBTreeReadLeafFrame frame, IIndexCursorStats stats, int index) {
+        this.frame = frame;
+        this.frameTuple = frame.createTupleReference(index, this);
+        this.reusablePredicate = new RangePredicate();
+        this.stats = stats;
+        fileId = -1;
+        pageId = IBufferCache.INVALID_PAGEID;
+    }
+
+    @Override
+    public void doDestroy() throws HyracksDataException {
+        // No Op all resources are released in the close call
+    }
+
+    @Override
+    public ITupleReference doGetTuple() {
+        return frameTuple;
+    }
+
+    private void fetchNextLeafPage(int leafPage) throws HyracksDataException {
+        int nextLeafPage = leafPage;
+        do {
+            ICachedPage nextLeaf = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeafPage), false);
+            stats.getPageCounter().update(1);
+            bufferCache.unpin(page0);
+            page0 = nextLeaf;
+            frame.setPage(page0);
+            frameTuple.reset(0);
+            nextLeafPage = frame.getNextLeaf();
+        } while (frame.getTupleCount() == 0 && nextLeafPage > 0);
+    }
+
+    @Override
+    public boolean doHasNext() throws HyracksDataException {
+        int nextLeafPage;
+        if (frameTuple.isConsumed() && !firstNextCall) {
+            frameTuple.lastTupleReached();
+            nextLeafPage = frame.getNextLeaf();
+            if (nextLeafPage >= 0) {
+                fetchNextLeafPage(nextLeafPage);
+            } else {
+                return false;
+            }
+        }
+        return isNextIncluded();
+    }
+
+    @Override
+    public void doNext() throws HyracksDataException {
+        //NoOp
+    }
+
+    @Override
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        // in case open is called multiple times without closing
+        if (page0 != null) {
+            releasePages();
+        }
+        originalKeyCmp = initialState.getOriginalKeyComparator();
+        page0 = initialState.getPage();
+        pageId = ((BTreeCursorInitialState) initialState).getPageId();
+        frame.setPage(page0);
+        frame.setMultiComparator(originalKeyCmp);
+        frameTuple.reset(0);
+        initCursorPosition(searchPred);
+    }
+
+    protected void initCursorPosition(ISearchPredicate searchPred) throws HyracksDataException {
+        pred = (RangePredicate) searchPred;
+        lowKey = pred.getLowKey();
+        highKey = pred.getHighKey();
+
+        reusablePredicate.setLowKeyComparator(originalKeyCmp);
+        reusablePredicate.setHighKeyComparator(pred.getHighKeyComparator());
+        reusablePredicate.setHighKey(pred.getHighKey(), pred.isHighKeyInclusive());
+        firstNextCall = true;
+        advanceTupleToLowKey();
+    }
+
+    protected boolean isNextIncluded() throws HyracksDataException {
+        if (firstNextCall) {
+            //The first call of frameTuple.next() was done during the opening of the cursor
+            firstNextCall = false;
+            return true;
+        } else if (frameTuple.isConsumed()) {
+            //All tuple were consumed
+            return false;
+        }
+        //Next tuple
+        frameTuple.next();
+        //Check whether the frameTuple is not consumed and also include the search key
+        return highKey == null || isLessOrEqual(frameTuple, highKey, pred.isHighKeyInclusive());
+    }
+
+    protected void advanceTupleToLowKey() throws HyracksDataException {
+        if (highKey != null && isLessOrEqual(highKey, frame.getLeftmostTuple(), !pred.isHighKeyInclusive())) {
+            /*
+             * Lowest key from the frame is greater than the requested highKey. No tuple will satisfy the search
+             * key. Consume the frameTuple to stop the search
+             */
+            firstNextCall = false;
+            frameTuple.consume();
+            return;
+        } else if (lowKey == null) {
+            //No range was specified.
+            frameTuple.next();
+            return;
+        }
+
+        //The lowKey is somewhere within the frame tuples
+        boolean stop = false;
+        int counter = 0;
+        while (!stop && !frameTuple.isConsumed()) {
+            frameTuple.next();
+            stop = isLessOrEqual(lowKey, frameTuple, pred.isLowKeyInclusive());
+            counter++;
+        }
+        //Advance all columns to the proper position
+        frameTuple.skip(counter - 1);
+    }
+
+    protected void releasePages() throws HyracksDataException {
+        //Unpin all column pages first
+        frameTuple.unpinColumnsPages();
+        if (page0 != null) {
+            bufferCache.unpin(page0);
+        }
+    }
+
+    private boolean isLessOrEqual(ITupleReference left, ITupleReference right, boolean inclusive)
+            throws HyracksDataException {
+        int cmp = originalKeyCmp.compare(left, right);
+        return cmp < 0 || cmp == 0 && inclusive;
+    }
+
+    @Override
+    public void doClose() throws HyracksDataException {
+        releasePages();
+        page0 = null;
+        pred = null;
+    }
+
+    @Override
+    public void setBufferCache(IBufferCache bufferCache) {
+        this.bufferCache = bufferCache;
+    }
+
+    @Override
+    public void setFileId(int fileId) {
+        this.fileId = fileId;
+    }
+
+    @Override
+    public boolean isExclusiveLatchNodes() {
+        return false;
+    }
+
+    /*
+     * ***********************************************************
+     * IColumnReadMultiPageOp
+     * ***********************************************************
+     */
+    @Override
+    public ICachedPage pin(int pageId) throws HyracksDataException {
+        stats.getPageCounter().update(1);
+        return bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+    }
+
+    @Override
+    public void unpin(ICachedPage page) throws HyracksDataException {
+        bufferCache.unpin(page);
+    }
+
+    @Override
+    public int getPageSize() {
+        return bufferCache.getPageSize();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeReadLeafFrame.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeReadLeafFrame.java
new file mode 100644
index 0000000..8872613
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeReadLeafFrame.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.impls.btree;
+
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleReader;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnReadMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.common.buffercache.CachedPage;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+
+public final class ColumnBTreeReadLeafFrame extends AbstractColumnBTreeLeafFrame {
+    private final AbstractColumnTupleReader columnarTupleReader;
+    private final ITreeIndexTupleReference leftMostTuple;
+    private final ITreeIndexTupleReference rightMostTuple;
+
+    public ColumnBTreeReadLeafFrame(ITreeIndexTupleWriter rowTupleWriter,
+            AbstractColumnTupleReader columnarTupleReader) {
+        super(rowTupleWriter);
+        this.columnarTupleReader = columnarTupleReader;
+        leftMostTuple = rowTupleWriter.createTupleReference();
+        rightMostTuple = rowTupleWriter.createTupleReference();
+    }
+
+    @Override
+    public ITupleReference getLeftmostTuple() {
+        if (getTupleCount() == 0) {
+            return null;
+        }
+        leftMostTuple.setFieldCount(cmp.getKeyFieldCount());
+        leftMostTuple.resetByTupleOffset(buf.array(), buf.getInt(LEFT_MOST_KEY_OFFSET));
+        return leftMostTuple;
+    }
+
+    @Override
+    public ITupleReference getRightmostTuple() {
+        if (getTupleCount() == 0) {
+            return null;
+        }
+        rightMostTuple.setFieldCount(cmp.getKeyFieldCount());
+        rightMostTuple.resetByTupleOffset(buf.array(), buf.getInt(RIGHT_MOST_KEY_OFFSET));
+        return rightMostTuple;
+    }
+
+    public IColumnTupleIterator createTupleReference(int index, IColumnReadMultiPageOp multiPageOp) {
+        return columnarTupleReader.createTupleIterator(this, index, multiPageOp);
+    }
+
+    @Override
+    public int getTupleCount() {
+        return buf.getInt(Constants.TUPLE_COUNT_OFFSET);
+    }
+
+    public int getPageId() {
+        return BufferedFileHandle.getPageId(((CachedPage) page).getDiskPageId());
+    }
+
+    public int getNumberOfColumns() {
+        return buf.getInt(NUMBER_OF_COLUMNS_OFFSET);
+    }
+
+    public int getColumnOffset(int columnIndex) {
+        if (columnIndex >= getNumberOfColumns()) {
+            throw new IndexOutOfBoundsException(columnIndex + " >= " + getNumberOfColumns());
+        }
+        return columnarTupleReader.getColumnOffset(buf, columnIndex);
+    }
+
+    AbstractColumnTupleReader getColumnarTupleReader() {
+        return columnarTupleReader;
+    }
+
+    int getNextLeaf() {
+        return buf.getInt(NEXT_LEAF_OFFSET);
+    }
+
+    @Override
+    public ITreeIndexTupleReference createTupleReference() {
+        throw new IllegalArgumentException("Use createTupleReference(int)");
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeWriteLeafFrame.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeWriteLeafFrame.java
new file mode 100644
index 0000000..275fb0e
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/btree/ColumnBTreeWriteLeafFrame.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.hyracks.storage.am.lsm.btree.column.impls.btree;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.AbstractColumnTupleWriter;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnWriteMultiPageOp;
+
+public class ColumnBTreeWriteLeafFrame extends AbstractColumnBTreeLeafFrame {
+    private final AbstractColumnTupleWriter columnTupleWriter;
+
+    public ColumnBTreeWriteLeafFrame(ITreeIndexTupleWriter rowTupleWriter,
+            AbstractColumnTupleWriter columnTupleWriter) {
+        super(rowTupleWriter);
+        this.columnTupleWriter = columnTupleWriter;
+    }
+
+    @Override
+    public void initBuffer(byte level) {
+        buf.putInt(TUPLE_COUNT_OFFSET, 0);
+        buf.put(Constants.LEVEL_OFFSET, level);
+        buf.putInt(NUMBER_OF_COLUMNS_OFFSET, 0);
+        buf.putInt(LEFT_MOST_KEY_OFFSET, -1);
+        buf.putInt(RIGHT_MOST_KEY_OFFSET, -1);
+        buf.putInt(SIZE_OF_COLUMNS_OFFSETS_OFFSET, 0);
+        buf.putInt(NUMBER_OF_COLUMN_PAGES, 0);
+        buf.put(FLAG_OFFSET, (byte) 0);
+        buf.putInt(NEXT_LEAF_OFFSET, -1);
+    }
+
+    void flush(AbstractColumnTupleWriter columnWriter, int numberOfTuples, IColumnWriteMultiPageOp multiPageOp,
+            ITupleReference minKey, ITupleReference maxKey) throws HyracksDataException {
+        //Prepare the space for writing the columns' information such as the primary keys
+        buf.position(HEADER_SIZE);
+        //Write the columns' information including the columns' offsets and the primary keys
+        columnWriter.flush(buf);
+
+        //Write min and max keys
+        int offset = buf.position();
+        buf.putInt(LEFT_MOST_KEY_OFFSET, offset);
+        offset += rowTupleWriter.writeTuple(minKey, buf.array(), offset);
+        buf.putInt(RIGHT_MOST_KEY_OFFSET, offset);
+        rowTupleWriter.writeTuple(maxKey, buf.array(), offset);
+
+        //Write page information
+        int numberOfColumns = columnWriter.getNumberOfColumns();
+        buf.putInt(TUPLE_COUNT_OFFSET, numberOfTuples);
+        buf.putInt(NUMBER_OF_COLUMNS_OFFSET, numberOfColumns);
+        buf.putInt(SIZE_OF_COLUMNS_OFFSETS_OFFSET, columnWriter.getColumnOffsetsSize());
+        buf.putInt(NUMBER_OF_COLUMN_PAGES, multiPageOp.getNumberOfPersistentBuffers());
+    }
+
+    public AbstractColumnTupleWriter getColumnTupleWriter() {
+        return columnTupleWriter;
+    }
+
+    void setNextLeaf(int pageId) {
+        buf.putInt(NEXT_LEAF_OFFSET, pageId);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTree.java
new file mode 100644
index 0000000..048d9de
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTree.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.impls.lsm;
+
+import java.util.List;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnManager;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnMetadata;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnTupleProjector;
+import org.apache.hyracks.storage.am.lsm.btree.column.utils.ColumnUtil;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeBatchPointSearchCursor;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeRangeSearchCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.IComponentMetadata;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
+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.ILSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+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.ILSMPageWriteCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor.ICursorFactory;
+import org.apache.hyracks.storage.common.IIndexAccessParameters;
+import org.apache.hyracks.storage.common.IIndexCursorStats;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.util.trace.ITracer;
+
+public class LSMColumnBTree extends LSMBTree {
+    private static final ICursorFactory CURSOR_FACTORY = LSMColumnBTreeSearchCursor::new;
+    private final IColumnManager columnManager;
+    private final ILSMDiskComponentFactory mergeComponentFactory;
+    /**
+     * This column metadata only used during flush and dataset bulkload operations. We cannot have more than one
+     * thread to do a flush/dataset bulkload. Do not use it for search/scan. Instead, use the latest component
+     * metadata of the operational disk components.
+     *
+     * @see LSMColumnBTreeOpContext#createProjectionInfo()
+     */
+    private IColumnMetadata columnMetadata;
+
+    public LSMColumnBTree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
+            ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory insertLeafFrameFactory,
+            ITreeIndexFrameFactory deleteLeafFrameFactory, IBufferCache diskBufferCache,
+            ILSMIndexFileManager fileManager, ILSMDiskComponentFactory componentFactory,
+            ILSMDiskComponentFactory mergeComponentFactory, ILSMDiskComponentFactory bulkloadComponentFactory,
+            double bloomFilterFalsePositiveRate, int fieldCount, IBinaryComparatorFactory[] cmpFactories,
+            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+            int[] btreeFields, ITracer tracer, IColumnManager columnManager) throws HyracksDataException {
+        super(ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory, deleteLeafFrameFactory,
+                diskBufferCache, fileManager, componentFactory, bulkloadComponentFactory, null, null, null,
+                bloomFilterFalsePositiveRate, fieldCount, cmpFactories, mergePolicy, opTracker, ioScheduler,
+                ioOpCallbackFactory, pageWriteCallbackFactory, true, true, btreeFields, null, true, false, tracer);
+        this.columnManager = columnManager;
+        this.mergeComponentFactory = mergeComponentFactory;
+    }
+
+    @Override
+    public synchronized void activate() throws HyracksDataException {
+        super.activate();
+        if (diskComponents.isEmpty()) {
+            columnMetadata = columnManager.activate();
+        } else {
+            IComponentMetadata componentMetadata = diskComponents.get(0).getMetadata();
+            columnMetadata = columnManager.activate(ColumnUtil.getColumnMetadataCopy(componentMetadata));
+        }
+    }
+
+    @Override
+    public LSMColumnBTreeOpContext createOpContext(IIndexAccessParameters iap) {
+        int numBloomFilterKeyFields =
+                ((LSMColumnBTreeWithBloomFilterDiskComponentFactory) componentFactory).getBloomFilterKeyFields().length;
+        IColumnTupleProjector tupleProjector =
+                ColumnUtil.getTupleProjector(iap, columnManager.getMergeColumnProjector());
+        return new LSMColumnBTreeOpContext(this, memoryComponents, insertLeafFrameFactory, deleteLeafFrameFactory,
+                (IExtendedModificationOperationCallback) iap.getModificationCallback(),
+                iap.getSearchOperationCallback(), numBloomFilterKeyFields, getTreeFields(), getFilterFields(),
+                getHarness(), getFilterCmpFactories(), tracer, tupleProjector);
+    }
+
+    protected IColumnManager getColumnManager() {
+        return columnManager;
+    }
+
+    protected IColumnMetadata getColumnMetadata() {
+        return columnMetadata;
+    }
+
+    @Override
+    protected LSMBTreeRangeSearchCursor createCursor(AbstractLSMIndexOperationContext opCtx,
+            boolean returnDeletedTuples, IIndexCursorStats stats) {
+        return new LSMColumnBTreeRangeSearchCursor(opCtx, returnDeletedTuples, stats);
+    }
+
+    @Override
+    public LSMBTreeBatchPointSearchCursor createBatchPointSearchCursor(ILSMIndexOperationContext opCtx) {
+        return new LSMColumnBatchPointSearchCursor(opCtx);
+    }
+
+    @Override
+    protected ILSMDiskComponentFactory getMergeComponentFactory() {
+        return mergeComponentFactory;
+    }
+
+    @Override
+    public ICursorFactory getCursorFactory() {
+        return CURSOR_FACTORY;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeOpContext.java
new file mode 100644
index 0000000..8a33de1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeOpContext.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.impls.lsm;
+
+import java.util.List;
+
+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.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnTupleProjector;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm.tuples.ColumnAwareMultiComparator;
+import org.apache.hyracks.storage.am.lsm.btree.column.utils.ColumnUtil;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeOpContext;
+import org.apache.hyracks.storage.am.lsm.common.api.IComponentMetadata;
+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.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.common.ISearchOperationCallback;
+import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.util.trace.ITracer;
+
+public class LSMColumnBTreeOpContext extends LSMBTreeOpContext {
+    private final IColumnTupleProjector projector;
+
+    public LSMColumnBTreeOpContext(ILSMIndex index, List<ILSMMemoryComponent> mutableComponents,
+            ITreeIndexFrameFactory insertLeafFrameFactory, ITreeIndexFrameFactory deleteLeafFrameFactory,
+            IExtendedModificationOperationCallback modificationCallback, ISearchOperationCallback searchCallback,
+            int numBloomFilterKeyFields, int[] btreeFields, int[] filterFields, ILSMHarness lsmHarness,
+            IBinaryComparatorFactory[] filterCmpFactories, ITracer tracer, IColumnTupleProjector projector) {
+        super(index, mutableComponents, insertLeafFrameFactory, deleteLeafFrameFactory, modificationCallback,
+                searchCallback, numBloomFilterKeyFields, btreeFields, filterFields, lsmHarness, filterCmpFactories,
+                tracer);
+        this.projector = projector;
+    }
+
+    public IColumnProjectionInfo createProjectionInfo() throws HyracksDataException {
+        List<ILSMComponent> operationalComponents = getComponentHolder();
+        IComponentMetadata componentMetadata = null;
+        for (int i = 0; i < operationalComponents.size() && componentMetadata == null; i++) {
+            ILSMComponent component = operationalComponents.get(i);
+            if (component.getType() == LSMComponentType.DISK) {
+                //Find the first on-disk component, which has the most recent column metadata.
+                componentMetadata = component.getMetadata();
+            }
+        }
+        if (componentMetadata != null) {
+            IValueReference columnMetadata = ColumnUtil.getColumnMetadataCopy(componentMetadata);
+            return projector.createProjectionInfo(columnMetadata);
+        }
+        //In-memory components only
+        return null;
+    }
+
+    @Override
+    protected MultiComparator createMultiComparator(IBinaryComparatorFactory[] cmpFactories) {
+        IBinaryComparator[] comparators = new IBinaryComparator[cmpFactories.length];
+        for (int i = 0; i < comparators.length; i++) {
+            comparators[i] = cmpFactories[i].createBinaryComparator();
+        }
+        return new ColumnAwareMultiComparator(comparators);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeRangeSearchCursor.java
new file mode 100644
index 0000000..75b2c72
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeRangeSearchCursor.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.impls.lsm;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.BTree;
+import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTree;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeRangeSearchCursor;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm.tuples.ColumnAwareDiskOnlyMultiComparator;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeRangeSearchCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.IIndexCursorStats;
+import org.apache.hyracks.storage.common.NoOpIndexCursorStats;
+
+public class LSMColumnBTreeRangeSearchCursor extends LSMBTreeRangeSearchCursor {
+    private final List<IColumnTupleIterator> componentTupleList;
+
+    public LSMColumnBTreeRangeSearchCursor(ILSMIndexOperationContext opCtx) {
+        this(opCtx, false, NoOpIndexCursorStats.INSTANCE);
+    }
+
+    public LSMColumnBTreeRangeSearchCursor(ILSMIndexOperationContext opCtx, boolean returnDeletedTuples,
+            IIndexCursorStats stats) {
+        super(opCtx, returnDeletedTuples, stats);
+        componentTupleList = new ArrayList<>();
+    }
+
+    @Override
+    protected BTreeAccessor createAccessor(LSMComponentType type, BTree btree, int index) throws HyracksDataException {
+        if (type == LSMComponentType.MEMORY) {
+            return super.createAccessor(type, btree, index);
+        }
+        ColumnBTree columnBTree = (ColumnBTree) btree;
+        LSMColumnBTreeOpContext columnOpCtx = (LSMColumnBTreeOpContext) opCtx;
+        return columnBTree.createAccessor(iap, index, columnOpCtx.createProjectionInfo());
+    }
+
+    @Override
+    protected IIndexCursor createCursor(LSMComponentType type, BTreeAccessor accessor) {
+        if (type == LSMComponentType.MEMORY) {
+            return super.createCursor(type, accessor);
+        }
+        ColumnBTreeRangeSearchCursor cursor = (ColumnBTreeRangeSearchCursor) accessor.createSearchCursor(false);
+        componentTupleList.add((IColumnTupleIterator) cursor.doGetTuple());
+        return cursor;
+    }
+
+    @Override
+    protected void markAsDeleted(PriorityQueueElement e) throws HyracksDataException {
+        if (isMemoryComponent[e.getCursorIndex()]) {
+            super.markAsDeleted(e);
+            return;
+        }
+        IColumnTupleIterator columnTuple = (IColumnTupleIterator) e.getTuple();
+        columnTuple.skip(1);
+    }
+
+    @Override
+    protected void setPriorityQueueComparator() {
+        if (!includeMutableComponent) {
+            cmp = new ColumnAwareDiskOnlyMultiComparator(cmp);
+        }
+        if (pqCmp == null || cmp != pqCmp.getMultiComparator()) {
+            pqCmp = new PriorityQueueComparator(cmp);
+        }
+    }
+
+    @Override
+    protected void excludeMemoryComponent() {
+        //Replace the comparator with disk only comparator
+        pqCmp.setMultiComparator(new ColumnAwareDiskOnlyMultiComparator(cmp));
+    }
+
+    @Override
+    protected int replaceFrom() {
+        //Disable replacing the in-memory component to disk component as the schema may change
+        //TODO at least allow the replacement when no schema changes occur
+        return -1;
+    }
+
+    /**
+     * @return we need the tuple references for vertical merges
+     */
+    public List<IColumnTupleIterator> getComponentTupleList() {
+        return componentTupleList;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeSearchCursor.java
new file mode 100644
index 0000000..6ca41d0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeSearchCursor.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm;
+
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeDiskComponentScanCursor;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeSearchCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+
+public class LSMColumnBTreeSearchCursor extends LSMBTreeSearchCursor {
+    public LSMColumnBTreeSearchCursor(ILSMIndexOperationContext opCtx) {
+        super(new LSMColumnPointSearchCursor(opCtx), new LSMColumnBTreeRangeSearchCursor(opCtx),
+                new LSMBTreeDiskComponentScanCursor(opCtx));
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeWithBloomFilterDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeWithBloomFilterDiskComponent.java
new file mode 100644
index 0000000..57e162d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeWithBloomFilterDiskComponent.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.hyracks.storage.am.lsm.btree.column.impls.lsm;
+
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import org.apache.hyracks.storage.am.btree.impls.BTree;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnMetadata;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTree;
+import org.apache.hyracks.storage.am.lsm.btree.column.utils.ColumnUtil;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeMergeOperation;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBloomFilterDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.IComponentMetadata;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
+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.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.ChainedLSMDiskComponentBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.impls.IChainedComponentBulkLoader;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+
+public class LSMColumnBTreeWithBloomFilterDiskComponent extends LSMBTreeWithBloomFilterDiskComponent {
+
+    public LSMColumnBTreeWithBloomFilterDiskComponent(AbstractLSMIndex lsmIndex, BTree btree, BloomFilter bloomFilter,
+            ILSMComponentFilter filter) {
+        super(lsmIndex, btree, bloomFilter, filter);
+    }
+
+    @Override
+    public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
+            boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
+            boolean cleanupEmptyComponent, IPageWriteCallback callback) throws HyracksDataException {
+        ChainedLSMDiskComponentBulkLoader chainedBulkLoader =
+                new ChainedLSMDiskComponentBulkLoader(operation, this, cleanupEmptyComponent);
+        if (withFilter && getLsmIndex().getFilterFields() != null) {
+            //Add filter writer if exists
+            chainedBulkLoader.addBulkLoader(createFilterBulkLoader());
+        }
+        //Add index bulkloader
+        chainedBulkLoader.addBulkLoader(createColumnIndexBulkLoader(operation, fillFactor, verifyInput, callback));
+
+        if (numElementsHint > 0) {
+            chainedBulkLoader.addBulkLoader(createBloomFilterBulkLoader(numElementsHint, callback));
+        }
+
+        callback.initialize(chainedBulkLoader);
+        return chainedBulkLoader;
+    }
+
+    private IChainedComponentBulkLoader createColumnIndexBulkLoader(ILSMIOOperation operation, float fillFactor,
+            boolean verifyInput, IPageWriteCallback callback) throws HyracksDataException {
+        LSMIOOperationType operationType = operation.getIOOpertionType();
+        LSMColumnBTree lsmColumnBTree = (LSMColumnBTree) getLsmIndex();
+        ColumnBTree columnBTree = (ColumnBTree) getIndex();
+        IColumnMetadata columnMetadata;
+        if (operationType == LSMIOOperationType.FLUSH || operationType == LSMIOOperationType.LOAD) {
+            columnMetadata = lsmColumnBTree.getColumnMetadata();
+        } else {
+            //Merge
+            LSMBTreeMergeOperation mergeOp = (LSMBTreeMergeOperation) operation;
+            LSMColumnBTreeRangeSearchCursor cursor = (LSMColumnBTreeRangeSearchCursor) mergeOp.getCursor();
+            List<ILSMComponent> mergingComponents = mergeOp.getMergingComponents();
+            IComponentMetadata componentMetadata = mergingComponents.get(0).getMetadata();
+            IValueReference columnMetadataValue = ColumnUtil.getColumnMetadataCopy(componentMetadata);
+            columnMetadata = lsmColumnBTree.getColumnManager().createMergeColumnMetadata(columnMetadataValue,
+                    cursor.getComponentTupleList());
+        }
+        IIndexBulkLoader bulkLoader = columnBTree.createBulkLoader(fillFactor, verifyInput, callback, columnMetadata);
+        return new LSMColumnIndexBulkloader(bulkLoader, columnMetadata, getMetadata());
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeWithBloomFilterDiskComponentFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeWithBloomFilterDiskComponentFactory.java
new file mode 100644
index 0000000..2d86a73
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBTreeWithBloomFilterDiskComponentFactory.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.hyracks.storage.am.lsm.btree.column.impls.lsm;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTree;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBloomFilterDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+
+/**
+ * We only create a BTree with a bloom filter
+ */
+public class LSMColumnBTreeWithBloomFilterDiskComponentFactory implements ILSMDiskComponentFactory {
+    private final TreeIndexFactory<ColumnBTree> btreeFactory;
+    private final BloomFilterFactory bloomFilterFactory;
+
+    public LSMColumnBTreeWithBloomFilterDiskComponentFactory(TreeIndexFactory<ColumnBTree> btreeFactory,
+            BloomFilterFactory bloomFilterFactory) {
+        this.btreeFactory = btreeFactory;
+        this.bloomFilterFactory = bloomFilterFactory;
+    }
+
+    @Override
+    public LSMBTreeWithBloomFilterDiskComponent createComponent(AbstractLSMIndex lsmIndex,
+            LSMComponentFileReferences cfr) throws HyracksDataException {
+        return new LSMColumnBTreeWithBloomFilterDiskComponent(lsmIndex,
+                btreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
+                bloomFilterFactory.createBloomFiltertInstance(cfr.getBloomFilterFileReference()), null);
+    }
+
+    public int[] getBloomFilterKeyFields() {
+        return bloomFilterFactory.getBloomFilterKeyFields();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBatchPointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBatchPointSearchCursor.java
new file mode 100644
index 0000000..65b292b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnBatchPointSearchCursor.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.hyracks.storage.am.lsm.btree.column.impls.lsm;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.BTree;
+import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTree;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeBatchPointSearchCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+
+public class LSMColumnBatchPointSearchCursor extends LSMBTreeBatchPointSearchCursor {
+
+    public LSMColumnBatchPointSearchCursor(ILSMIndexOperationContext opCtx) {
+        super(opCtx);
+    }
+
+    @Override
+    protected BTreeAccessor createAccessor(LSMComponentType type, BTree btree, int index) throws HyracksDataException {
+        if (type == LSMComponentType.MEMORY) {
+            return super.createAccessor(type, btree, index);
+        }
+        ColumnBTree columnBTree = (ColumnBTree) btree;
+        LSMColumnBTreeOpContext columnOpCtx = (LSMColumnBTreeOpContext) opCtx;
+        return columnBTree.createAccessor(NoOpIndexAccessParameters.INSTANCE, index,
+                columnOpCtx.createProjectionInfo());
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnIndexBulkloader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnIndexBulkloader.java
new file mode 100644
index 0000000..ba41227
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnIndexBulkloader.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.hyracks.storage.am.lsm.btree.column.impls.lsm;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnMetadata;
+import org.apache.hyracks.storage.am.lsm.btree.column.utils.ColumnUtil;
+import org.apache.hyracks.storage.am.lsm.common.api.IComponentMetadata;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexBulkLoader;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+
+public class LSMColumnIndexBulkloader extends LSMIndexBulkLoader {
+    private final IColumnMetadata columnMetadata;
+    private final IComponentMetadata componentMetadata;
+
+    public LSMColumnIndexBulkloader(IIndexBulkLoader bulkLoader, IColumnMetadata columnMetadata,
+            IComponentMetadata componentMetadata) {
+        super(bulkLoader);
+        this.columnMetadata = columnMetadata;
+        this.componentMetadata = componentMetadata;
+    }
+
+    @Override
+    public void end() throws HyracksDataException {
+        ColumnUtil.putColumnsMetadataValue(columnMetadata.serializeColumnsMetadata(), componentMetadata);
+        super.end();
+    }
+
+    @Override
+    public void abort() throws HyracksDataException {
+        columnMetadata.abort();
+        super.abort();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnPointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnPointSearchCursor.java
new file mode 100644
index 0000000..e193232
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/LSMColumnPointSearchCursor.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.hyracks.storage.am.lsm.btree.column.impls.lsm;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.BTree;
+import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTree;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreePointSearchCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+
+public class LSMColumnPointSearchCursor extends LSMBTreePointSearchCursor {
+
+    public LSMColumnPointSearchCursor(ILSMIndexOperationContext opCtx) {
+        super(opCtx);
+    }
+
+    @Override
+    protected BTreeAccessor createAccessor(LSMComponentType type, BTree btree, int index) throws HyracksDataException {
+        if (type == LSMComponentType.MEMORY) {
+            return super.createAccessor(type, btree, index);
+        }
+        ColumnBTree columnBTree = (ColumnBTree) btree;
+        LSMColumnBTreeOpContext columnOpCtx = (LSMColumnBTreeOpContext) opCtx;
+        return columnBTree.createAccessor(NoOpIndexAccessParameters.INSTANCE, index,
+                columnOpCtx.createProjectionInfo());
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/AbstractColumnTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/AbstractColumnTupleReference.java
new file mode 100644
index 0000000..5a3b111
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/AbstractColumnTupleReference.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.impls.lsm.tuples;
+
+import static org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.AbstractColumnBTreeLeafFrame.HEADER_SIZE;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnBufferProvider;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnReadMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnProjectionInfo;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+
+public abstract class AbstractColumnTupleReference implements IColumnTupleIterator {
+    private static final String UNSUPPORTED_OPERATION_MSG = "Operation is not supported for column tuples";
+    private final int componentIndex;
+    private final ColumnBTreeReadLeafFrame frame;
+    private final IColumnBufferProvider[] primaryKeyBufferProviders;
+    private final IColumnBufferProvider[] buffersProviders;
+    private final int numberOfPrimaryKeys;
+    protected int tupleIndex;
+
+    /**
+     * Column tuple reference
+     *
+     * @param componentIndex LSM component index
+     * @param frame          page0 frame
+     * @param info           projection info
+     */
+    protected AbstractColumnTupleReference(int componentIndex, ColumnBTreeReadLeafFrame frame,
+            IColumnProjectionInfo info, IColumnReadMultiPageOp multiPageOp) {
+        this.componentIndex = componentIndex;
+        this.frame = frame;
+        numberOfPrimaryKeys = info.getNumberOfPrimaryKeys();
+
+        primaryKeyBufferProviders = new IColumnBufferProvider[numberOfPrimaryKeys];
+
+        for (int i = 0; i < numberOfPrimaryKeys; i++) {
+            primaryKeyBufferProviders[i] = new ColumnSingleBufferProvider(i);
+        }
+
+        int numberOfRequestedColumns = info.getNumberOfProjectedColumns();
+        buffersProviders = new IColumnBufferProvider[numberOfRequestedColumns];
+        for (int i = 0; i < numberOfRequestedColumns; i++) {
+            int columnIndex = info.getColumnIndex(i);
+            if (columnIndex >= numberOfPrimaryKeys) {
+                buffersProviders[i] = new ColumnMultiBufferProvider(columnIndex, multiPageOp);
+            } else {
+                buffersProviders[i] = new ColumnSingleBufferProvider(columnIndex);
+            }
+        }
+    }
+
+    @Override
+    public final void reset(int startIndex) throws HyracksDataException {
+        tupleIndex = startIndex;
+        ByteBuffer pageZero = frame.getBuffer();
+        pageZero.clear();
+        pageZero.position(HEADER_SIZE);
+
+        int numberOfTuples = frame.getTupleCount();
+        //Start new page and check whether we should skip reading non-key columns or not
+        boolean readColumnPages = startNewPage(pageZero, frame.getNumberOfColumns(), numberOfTuples);
+
+        //Start primary keys
+        for (int i = 0; i < numberOfPrimaryKeys; i++) {
+            IColumnBufferProvider provider = primaryKeyBufferProviders[i];
+            provider.reset(frame);
+            startPrimaryKey(provider, tupleIndex, i, numberOfTuples);
+        }
+
+        if (readColumnPages) {
+            for (int i = 0; i < buffersProviders.length; i++) {
+                IColumnBufferProvider provider = buffersProviders[i];
+                //Release previous pinned pages if any
+                provider.releaseAll();
+                provider.reset(frame);
+                startColumn(provider, tupleIndex, i, numberOfTuples);
+            }
+        }
+    }
+
+    protected abstract boolean startNewPage(ByteBuffer pageZero, int numberOfColumns, int numberOfTuples);
+
+    protected abstract void startPrimaryKey(IColumnBufferProvider bufferProvider, int startIndex, int ordinal,
+            int numberOfTuples) throws HyracksDataException;
+
+    protected abstract void startColumn(IColumnBufferProvider buffersProvider, int startIndex, int ordinal,
+            int numberOfTuples) throws HyracksDataException;
+
+    protected abstract void onNext() throws HyracksDataException;
+
+    @Override
+    public final void next() throws HyracksDataException {
+        onNext();
+        tupleIndex++;
+    }
+
+    @Override
+    public final void consume() {
+        tupleIndex = frame.getTupleCount();
+    }
+
+    @Override
+    public final boolean isConsumed() {
+        return tupleIndex >= frame.getTupleCount();
+    }
+
+    @Override
+    public final int getComponentIndex() {
+        return componentIndex;
+    }
+
+    @Override
+    public final void unpinColumnsPages() throws HyracksDataException {
+        for (int i = 0; i < buffersProviders.length; i++) {
+            buffersProviders[i].releaseAll();
+        }
+    }
+
+    /* *************************************************************
+     * Unsupported Operations
+     * *************************************************************
+     */
+
+    @Override
+    public final void setFieldCount(int fieldCount) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void setFieldCount(int fieldStartIndex, int fieldCount) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void resetByTupleOffset(byte[] buf, int tupleStartOffset) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+
+    @Override
+    public final void resetByTupleIndex(ITreeIndexFrame frame, int tupleIndex) {
+        throw new UnsupportedOperationException(UNSUPPORTED_OPERATION_MSG);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnAwareDiskOnlyMultiComparator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnAwareDiskOnlyMultiComparator.java
new file mode 100644
index 0000000..297b740
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnAwareDiskOnlyMultiComparator.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm.tuples;
+
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.common.MultiComparator;
+
+public class ColumnAwareDiskOnlyMultiComparator extends MultiComparator {
+    public ColumnAwareDiskOnlyMultiComparator(MultiComparator comparator) {
+        super(comparator.getComparators());
+    }
+
+    @Override
+    public int compare(ITupleReference tupleA, ITupleReference tupleB) {
+        return ((IColumnTupleIterator) tupleA).compareTo((IColumnTupleIterator) tupleB);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnAwareMultiComparator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnAwareMultiComparator.java
new file mode 100644
index 0000000..0438f41
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnAwareMultiComparator.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.hyracks.storage.am.lsm.btree.column.impls.lsm.tuples;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnTupleIterator;
+import org.apache.hyracks.storage.common.MultiComparator;
+
+public class ColumnAwareMultiComparator extends MultiComparator {
+    public ColumnAwareMultiComparator(IBinaryComparator[] cmps) {
+        super(cmps);
+    }
+
+    @Override
+    public int compare(ITupleReference tupleA, ITupleReference tupleB) throws HyracksDataException {
+        if (tupleA instanceof IColumnTupleIterator && tupleB instanceof IColumnTupleIterator) {
+            //Avoid comparing serialized data
+            return ((IColumnTupleIterator) tupleA).compareTo((IColumnTupleIterator) tupleB);
+        }
+        return super.compare(tupleA, tupleB);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnMultiBufferProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnMultiBufferProvider.java
new file mode 100644
index 0000000..0c17d6b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnMultiBufferProvider.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.impls.lsm.tuples;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnBufferProvider;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnReadMultiPageOp;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+
+public final class ColumnMultiBufferProvider implements IColumnBufferProvider {
+    private final int columnIndex;
+    private final IColumnReadMultiPageOp multiPageOp;
+    private final Queue<ICachedPage> pages;
+    private int numberOfPages;
+    private int startPage;
+    private int startOffset;
+    private int length;
+
+    public ColumnMultiBufferProvider(int columnIndex, IColumnReadMultiPageOp multiPageOp) {
+        this.columnIndex = columnIndex;
+        this.multiPageOp = multiPageOp;
+        pages = new ArrayDeque<>();
+    }
+
+    @Override
+    public void reset(ColumnBTreeReadLeafFrame frame) throws HyracksDataException {
+        if (columnIndex >= frame.getNumberOfColumns()) {
+            numberOfPages = 0;
+            length = 0;
+            return;
+        }
+
+        int offset = frame.getColumnOffset(columnIndex);
+        startPage = frame.getPageId() + getColumnPageIndex(offset);
+        startOffset = offset % multiPageOp.getPageSize();
+        //Duplicate as the buffer could be shared by more than one column
+        ByteBuffer firstPage = readNext().duplicate();
+        firstPage.position(startOffset);
+        //Read the length
+        length = firstPage.getInt();
+        int remainingLength = length - firstPage.remaining();
+        numberOfPages = (int) Math.ceil((double) remainingLength / multiPageOp.getPageSize());
+        //+4-bytes after reading the length
+        startOffset += Integer.BYTES;
+        //-4-bytes after reading the length
+        length -= Integer.BYTES;
+    }
+
+    @Override
+    public void readAll(Queue<ByteBuffer> buffers) throws HyracksDataException {
+        ByteBuffer buffer = pages.peek().getBuffer().duplicate();
+        buffer.clear();
+        buffer.position(startOffset);
+        buffers.add(buffer);
+        for (int i = 0; i < numberOfPages; i++) {
+            buffer = readNext().duplicate();
+            buffer.clear();
+            buffers.add(buffer);
+        }
+        numberOfPages = 0;
+    }
+
+    @Override
+    public void releaseAll() throws HyracksDataException {
+        while (!pages.isEmpty()) {
+            ICachedPage page = pages.poll();
+            multiPageOp.unpin(page);
+        }
+    }
+
+    @Override
+    public int getLength() {
+        return length;
+    }
+
+    @Override
+    public ByteBuffer getBuffer() {
+        throw new UnsupportedOperationException("Use readAll() for multi-buffer");
+    }
+
+    @Override
+    public int getColumnIndex() {
+        return columnIndex;
+    }
+
+    private ByteBuffer readNext() throws HyracksDataException {
+        ICachedPage columnPage = multiPageOp.pin(startPage++);
+        pages.add(columnPage);
+        return columnPage.getBuffer();
+    }
+
+    private int getColumnPageIndex(int columnOffset) {
+        return (int) Math.floor((double) columnOffset / multiPageOp.getPageSize());
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnSingleBufferProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnSingleBufferProvider.java
new file mode 100644
index 0000000..3ae5c7d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/impls/lsm/tuples/ColumnSingleBufferProvider.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.column.impls.lsm.tuples;
+
+import java.nio.ByteBuffer;
+import java.util.Queue;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnBufferProvider;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeReadLeafFrame;
+
+/**
+ * Buffer provider for Primary Keys
+ */
+public final class ColumnSingleBufferProvider implements IColumnBufferProvider {
+    private final int columnIndex;
+    private ByteBuffer buffer;
+
+    public ColumnSingleBufferProvider(int columnIndex) {
+        this.columnIndex = columnIndex;
+    }
+
+    @Override
+    public void reset(ColumnBTreeReadLeafFrame frame) {
+        int offset = frame.getColumnOffset(columnIndex);
+        this.buffer = frame.getBuffer().duplicate();
+        buffer.position(offset);
+    }
+
+    @Override
+    public void readAll(Queue<ByteBuffer> buffers) {
+        throw new UnsupportedOperationException("Use getBuffer() for single-buffer");
+    }
+
+    @Override
+    public void releaseAll() throws HyracksDataException {
+        //NoOp
+    }
+
+    @Override
+    public ByteBuffer getBuffer() {
+        return buffer;
+    }
+
+    @Override
+    public int getLength() {
+        return -1;
+    }
+
+    @Override
+    public int getColumnIndex() {
+        return columnIndex;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/utils/ColumnUtil.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/utils/ColumnUtil.java
new file mode 100644
index 0000000..25ed3db
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/utils/ColumnUtil.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.btree.column.utils;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.HyracksConstants;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.storage.am.common.freepage.MutableArrayValueReference;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnManager;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.projection.IColumnTupleProjector;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm.LSMColumnBTree;
+import org.apache.hyracks.storage.am.lsm.common.api.IComponentMetadata;
+import org.apache.hyracks.storage.common.IIndexAccessParameters;
+
+public class ColumnUtil {
+    /**
+     * Used to get the columns info from {@link IComponentMetadata#get(IValueReference, ArrayBackedValueStorage)}
+     *
+     * @see LSMColumnBTree#activate()
+     * @see IColumnManager#activate(IValueReference)
+     */
+    private static final MutableArrayValueReference COLUMNS_METADATA_KEY =
+            new MutableArrayValueReference("COLUMNS_METADATA".getBytes());
+
+    private ColumnUtil() {
+    }
+
+    public static IValueReference getColumnMetadataCopy(IComponentMetadata src) throws HyracksDataException {
+        ArrayBackedValueStorage storage = new ArrayBackedValueStorage();
+        src.get(COLUMNS_METADATA_KEY, storage);
+        return storage;
+    }
+
+    public static void putColumnsMetadataValue(IValueReference columnsMetadataValue, IComponentMetadata dest)
+            throws HyracksDataException {
+        dest.put(COLUMNS_METADATA_KEY, columnsMetadataValue);
+    }
+
+    public static IColumnTupleProjector getTupleProjector(IIndexAccessParameters iap,
+            IColumnTupleProjector defaultProjector) {
+        IColumnTupleProjector projector =
+                iap.getParameter(HyracksConstants.TUPLE_PROJECTOR, IColumnTupleProjector.class);
+        return projector == null ? defaultProjector : projector;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/utils/LSMColumnBTreeUtil.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/utils/LSMColumnBTreeUtil.java
new file mode 100644
index 0000000..1a55447
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree-column/src/main/java/org/apache/hyracks/storage/am/lsm/btree/column/utils/LSMColumnBTreeUtil.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.column.utils;
+
+import java.util.List;
+
+import org.apache.hyracks.api.compression.ICompressorDecompressorFactory;
+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.api.io.FileReference;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
+import org.apache.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
+import org.apache.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.api.IColumnManagerFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTree;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.btree.ColumnBTreeLeafFrameFactory;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm.LSMColumnBTree;
+import org.apache.hyracks.storage.am.lsm.btree.column.impls.lsm.LSMColumnBTreeWithBloomFilterDiskComponentFactory;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeFileManager;
+import org.apache.hyracks.storage.am.lsm.btree.tuples.LSMBTreeCopyTupleWriterFactory;
+import org.apache.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleWriterFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
+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.ILSMIndexFileManager;
+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.ILSMPageWriteCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import org.apache.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.util.trace.ITracer;
+
+public class LSMColumnBTreeUtil {
+
+    public static LSMBTree createLSMTree(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
+            FileReference file, IBufferCache diskBufferCache, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
+            ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+            int[] btreeFields, IMetadataPageManagerFactory freePageManagerFactory, boolean updateAware, ITracer tracer,
+            ICompressorDecompressorFactory compressorDecompressorFactory, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector, IColumnManagerFactory columnManagerFactory)
+            throws HyracksDataException {
+
+        //Tuple writers
+        LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+                cmpFactories.length, false, updateAware, nullTypeTraits, nullIntrospector);
+        LSMBTreeTupleWriterFactory deleteTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+                cmpFactories.length, true, updateAware, nullTypeTraits, nullIntrospector);
+        LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory = new LSMBTreeCopyTupleWriterFactory(typeTraits,
+                cmpFactories.length, updateAware, nullTypeTraits, nullIntrospector);
+        LSMBTreeTupleWriterFactory bulkLoadTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+                cmpFactories.length, false, updateAware, nullTypeTraits, nullIntrospector);
+
+        //Leaf frames
+        ITreeIndexFrameFactory flushLeafFrameFactory = new ColumnBTreeLeafFrameFactory(copyTupleWriterFactory,
+                columnManagerFactory.getFlushColumnTupleReaderWriterFactory());
+        ITreeIndexFrameFactory mergeLeafFrameFactory = new ColumnBTreeLeafFrameFactory(copyTupleWriterFactory,
+                columnManagerFactory.createMergeColumnTupleReaderWriterFactory());
+        ITreeIndexFrameFactory bulkLoadLeafFrameFactory = new ColumnBTreeLeafFrameFactory(bulkLoadTupleWriterFactory,
+                columnManagerFactory.getLoadColumnTupleReaderWriterFactory());
+        ITreeIndexFrameFactory insertLeafFrameFactory = new BTreeNSMLeafFrameFactory(insertTupleWriterFactory);
+        ITreeIndexFrameFactory deleteLeafFrameFactory = new BTreeNSMLeafFrameFactory(deleteTupleWriterFactory);
+        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(insertTupleWriterFactory);
+
+        //BTree factory
+        TreeIndexFactory<ColumnBTree> flushBTreeFactory = new ColumnBTreeFactory(ioManager, diskBufferCache,
+                freePageManagerFactory, interiorFrameFactory, flushLeafFrameFactory, cmpFactories, typeTraits.length);
+        TreeIndexFactory<ColumnBTree> mergeBTreeFactory = new ColumnBTreeFactory(ioManager, diskBufferCache,
+                freePageManagerFactory, interiorFrameFactory, mergeLeafFrameFactory, cmpFactories, typeTraits.length);
+        TreeIndexFactory<ColumnBTree> bulkloadBTreeFactory =
+                new ColumnBTreeFactory(ioManager, diskBufferCache, freePageManagerFactory, interiorFrameFactory,
+                        bulkLoadLeafFrameFactory, cmpFactories, typeTraits.length);
+
+        ILSMIndexFileManager fileNameManager =
+                new LSMBTreeFileManager(ioManager, file, flushBTreeFactory, true, compressorDecompressorFactory);
+
+        BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, bloomFilterKeyFields);
+        ILSMDiskComponentFactory flushComponentFactory =
+                new LSMColumnBTreeWithBloomFilterDiskComponentFactory(flushBTreeFactory, bloomFilterFactory);
+        ILSMDiskComponentFactory mergeComponentFactory =
+                new LSMColumnBTreeWithBloomFilterDiskComponentFactory(mergeBTreeFactory, bloomFilterFactory);
+        ILSMDiskComponentFactory bulkLoadComponentFactory =
+                new LSMColumnBTreeWithBloomFilterDiskComponentFactory(bulkloadBTreeFactory, bloomFilterFactory);
+
+        return new LSMColumnBTree(ioManager, virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory,
+                deleteLeafFrameFactory, diskBufferCache, fileNameManager, flushComponentFactory, mergeComponentFactory,
+                bulkLoadComponentFactory, bloomFilterFalsePositiveRate, typeTraits.length, cmpFactories, mergePolicy,
+                opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, btreeFields, tracer,
+                columnManagerFactory.createColumnManager());
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
index e16baf9..ee4c4f7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
@@ -56,19 +56,21 @@
             ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             IMetadataPageManagerFactory metadataPageManagerFactory,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter,
-            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
+            boolean isSecondaryNoIncrementalMaintenance) {
         super(typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, path,
                 storageManager, mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories,
                 btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
                 metadataPageManagerFactory, null, ioSchedulerProvider, durable,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector);
+                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector,
+                isSecondaryNoIncrementalMaintenance);
     }
 
     private ExternalBTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
-            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter)
-            throws HyracksDataException {
+            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
+            boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
         super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter);
+                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
     }
 
     @Override
@@ -97,6 +99,6 @@
         LSMBTreeLocalResource lsmBtree = (LSMBTreeLocalResource) LSMBTreeLocalResource.fromJson(registry, json);
         return new ExternalBTreeLocalResource(registry, json, lsmBtree.bloomFilterKeyFields,
                 lsmBtree.bloomFilterFalsePositiveRate, lsmBtree.isPrimary, lsmBtree.btreeFields,
-                lsmBtree.hasBloomFilter);
+                lsmBtree.hasBloomFilter, lsmBtree.isSecondaryNoIncrementalMaintenance);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
index e4c29ba..3a28057 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
@@ -46,13 +46,13 @@
             IMetadataPageManagerFactory metadataPageManagerFactory,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
             Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
-            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
-            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+            double bloomFilterFalsePositiveRate, int[] btreeFields, boolean hasBloomFilter, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
                 ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, bloomFilterKeyFields,
-                bloomFilterFalsePositiveRate, isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
-                hasBloomFilter, nullTypeTraits, nullIntrospector);
+                bloomFilterFalsePositiveRate, false, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
+                hasBloomFilter, nullTypeTraits, nullIntrospector, true);
     }
 
     @Override
@@ -61,6 +61,7 @@
                 bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
                 mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, bloomFilterKeyFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector);
+                ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector,
+                isSecondaryNoIncrementalMaintenance);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
index 863ae56..f2c5263 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
@@ -59,19 +59,21 @@
             ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             IMetadataPageManagerFactory metadataPageManagerFactory,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter,
-            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
+            boolean isSecondaryNoIncrementalMaintenance) {
         super(typeTraits, cmpFactories, buddyBtreeFields, bloomFilterFalsePositiveRate, isPrimary, path, storageManager,
                 mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields,
                 filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
                 metadataPageManagerFactory, null, ioSchedulerProvider, durable,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector);
+                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector,
+                isSecondaryNoIncrementalMaintenance);
     }
 
     private ExternalBTreeWithBuddyLocalResource(IPersistedResourceRegistry registry, JsonNode json,
             int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
-            boolean hasBloomFilter) throws HyracksDataException {
+            boolean hasBloomFilter, boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
         super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter);
+                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
     }
 
     @Override
@@ -100,6 +102,6 @@
         LSMBTreeLocalResource lsmBtree = (LSMBTreeLocalResource) LSMBTreeLocalResource.fromJson(registry, json);
         return new ExternalBTreeWithBuddyLocalResource(registry, json, lsmBtree.bloomFilterKeyFields,
                 lsmBtree.bloomFilterFalsePositiveRate, lsmBtree.isPrimary, lsmBtree.btreeFields,
-                lsmBtree.hasBloomFilter);
+                lsmBtree.hasBloomFilter, lsmBtree.isSecondaryNoIncrementalMaintenance);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
index 39c8aed..995a23b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
@@ -46,13 +46,13 @@
             IMetadataPageManagerFactory metadataPageManagerFactory,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
             Map<String, String> mergePolicyProperties, boolean durable, int[] buddyBtreeFields,
-            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
-            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+            double bloomFilterFalsePositiveRate, int[] btreeFields, boolean hasBloomFilter, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
                 ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, buddyBtreeFields,
-                bloomFilterFalsePositiveRate, isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
-                hasBloomFilter, nullTypeTraits, nullIntrospector);
+                bloomFilterFalsePositiveRate, false, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
+                hasBloomFilter, nullTypeTraits, nullIntrospector, true);
     }
 
     @Override
@@ -61,6 +61,7 @@
                 bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
                 mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector);
+                ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector,
+                isSecondaryNoIncrementalMaintenance);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
index 3800d17..3e03e5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
@@ -29,6 +29,7 @@
 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.common.projection.ITupleProjectorFactory;
 
 public class LSMBTreeBatchPointSearchOperatorDescriptor extends BTreeSearchOperatorDescriptor {
     private static final long serialVersionUID = 1L;
@@ -38,10 +39,11 @@
             IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, ITupleFilterFactory tupleFilterFactory,
-            long outputLimit) {
+            long outputLimit, ITupleProjectorFactory tupleProjectorFactory) {
         super(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
-                maxFilterFieldIndexes, false, null, tupleFilterFactory, outputLimit, false, null, null);
+                maxFilterFieldIndexes, false, null, tupleFilterFactory, outputLimit, false, null, null,
+                tupleProjectorFactory);
     }
 
     @Override
@@ -51,7 +53,7 @@
                 recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), lowKeyFields, highKeyFields,
                 lowKeyInclusive, highKeyInclusive, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, tupleFilterFactory,
-                outputLimit);
+                outputLimit, tupleProjectorFactory);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
index 30813ef..f6f97b7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
@@ -34,10 +34,12 @@
 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.lsm.btree.impls.LSMBTree;
 import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeBatchPointSearchCursor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
 
 public class LSMBTreeBatchPointSearchOperatorNodePushable extends BTreeSearchOperatorNodePushable {
 
@@ -48,18 +50,19 @@
             boolean highKeyInclusive, int[] minFilterKeyFields, int[] maxFilterKeyFields,
             IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
-            ITupleFilterFactory tupleFilterFactory, long outputLimit) throws HyracksDataException {
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, ITupleProjectorFactory tupleProjectorFactory)
+            throws HyracksDataException {
         super(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
                 minFilterKeyFields, maxFilterKeyFields, indexHelperFactory, retainInput, retainMissing,
                 missingWriterFactory, searchCallbackFactory, false, null, tupleFilterFactory, outputLimit, false, null,
-                null);
+                null, tupleProjectorFactory);
         this.keyFields = lowKeyFields;
     }
 
     @Override
     protected IIndexCursor createCursor() throws HyracksDataException {
         ILSMIndexAccessor lsmAccessor = (ILSMIndexAccessor) indexAccessor;
-        return new LSMBTreeBatchPointSearchCursor(lsmAccessor.getOpContext());
+        return ((LSMBTree) index).createBatchPointSearchCursor(lsmAccessor.getOpContext());
     }
 
     @Override
@@ -123,7 +126,7 @@
                 break;
             }
         }
-        stats.getTupleCounter().update(matchingTupleCount);
+        stats.getInputTupleCounter().update(matchingTupleCount);
 
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
index 656ae02..a7e433c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
@@ -59,6 +59,7 @@
     protected final boolean isPrimary;
     protected final int[] btreeFields;
     protected final ICompressorDecompressorFactory compressorDecompressorFactory;
+    protected final boolean isSecondaryNoIncrementalMaintenance;
 
     public LSMBTreeLocalResource(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
             int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, boolean isPrimary, String path,
@@ -70,7 +71,8 @@
             IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable,
             ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter,
-            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
+            boolean isSecondaryNoIncrementalMaintenance) {
         super(path, storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
                 vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
@@ -81,12 +83,13 @@
         this.btreeFields = btreeFields;
         this.compressorDecompressorFactory = compressorDecompressorFactory;
         this.hasBloomFilter = hasBloomFilter;
+        this.isSecondaryNoIncrementalMaintenance = isSecondaryNoIncrementalMaintenance;
     }
 
     protected LSMBTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
             double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
-            ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter)
-            throws HyracksDataException {
+            ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter,
+            boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
         super(registry, json);
         this.bloomFilterKeyFields = bloomFilterKeyFields;
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
@@ -94,6 +97,7 @@
         this.btreeFields = btreeFields;
         this.compressorDecompressorFactory = compressorDecompressorFactory;
         this.hasBloomFilter = hasBloomFilter;
+        this.isSecondaryNoIncrementalMaintenance = isSecondaryNoIncrementalMaintenance;
     }
 
     @Override
@@ -114,6 +118,10 @@
                 compressorDecompressorFactory, hasBloomFilter, nullTypeTraits, nullIntrospector);
     }
 
+    public boolean isSecondaryNoIncrementalMaintenance() {
+        return isSecondaryNoIncrementalMaintenance;
+    }
+
     @Override
     public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
         final ObjectNode jsonObject = registry.getClassIdentifier(getClass(), serialVersionUID);
@@ -131,8 +139,10 @@
         final JsonNode compressorDecompressorNode = json.get("compressorDecompressorFactory");
         final ICompressorDecompressorFactory compDecompFactory = (ICompressorDecompressorFactory) registry
                 .deserializeOrDefault(compressorDecompressorNode, NoOpCompressorDecompressorFactory.class);
+        boolean isSecondaryNoIncrementalMaintenance =
+                getOrDefaultBoolean(json, "isSecondaryNoIncrementalMaintenance", false);
         return new LSMBTreeLocalResource(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary,
-                btreeFields, compDecompFactory, hasBloomFilter);
+                btreeFields, compDecompFactory, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
     }
 
     @Override
@@ -145,13 +155,16 @@
         json.put("isPrimary", isPrimary);
         json.putPOJO("btreeFields", btreeFields);
         json.putPOJO("compressorDecompressorFactory", compressorDecompressorFactory.toJson(registry));
+        json.put("isSecondaryNoIncrementalMaintenance", isSecondaryNoIncrementalMaintenance);
     }
 
     private static boolean getOrDefaultHasBloomFilter(JsonNode json, boolean isPrimary) {
-        if (json.has(HAS_BLOOM_FILTER_FIELD)) {
-            return json.get(HAS_BLOOM_FILTER_FIELD).asBoolean();
-        }
-        // for backward compatibiliy, only primary indexes have bloom filters
-        return isPrimary;
+        // for backward compatibility, only primary indexes have bloom filters
+        return getOrDefaultBoolean(json, HAS_BLOOM_FILTER_FIELD, isPrimary);
     }
+
+    private static boolean getOrDefaultBoolean(JsonNode jsonNode, String fieldName, boolean defaultValue) {
+        return jsonNode.has(fieldName) ? jsonNode.get(fieldName).asBoolean() : defaultValue;
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
index e926a48..6695e90 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
@@ -48,6 +48,7 @@
     protected final boolean isPrimary;
     protected final int[] btreeFields;
     protected final ICompressorDecompressorFactory compressorDecompressorFactory;
+    protected final boolean isSecondaryNoIncrementalMaintenance;
 
     public LSMBTreeLocalResourceFactory(IStorageManager storageManager, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
@@ -59,7 +60,8 @@
             Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
             double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
             ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter,
-            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
+            boolean isSecondaryNoIncrementalMaintenance) {
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
                 vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
@@ -70,6 +72,7 @@
         this.isPrimary = isPrimary;
         this.btreeFields = btreeFields;
         this.compressorDecompressorFactory = compressorDecompressorFactory;
+        this.isSecondaryNoIncrementalMaintenance = isSecondaryNoIncrementalMaintenance;
     }
 
     @Override
@@ -78,7 +81,8 @@
                 isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory, mergePolicyProperties,
                 filterTypeTraits, filterCmpFactories, btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory,
                 pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, durable,
-                compressorDecompressorFactory, hasBloomFilter, nullTypeTraits, nullIntrospector);
+                compressorDecompressorFactory, hasBloomFilter, nullTypeTraits, nullIntrospector,
+                isSecondaryNoIncrementalMaintenance);
     }
 
     private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
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 c3d1416..d2fbbef 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
@@ -341,7 +341,7 @@
                 try {
                     List<ILSMComponent> mergedComponents = mergeOp.getMergingComponents();
                     long numElements = getNumberOfElements(mergedComponents);
-                    mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(), null,
+                    mergedComponent = createDiskComponent(getMergeComponentFactory(), mergeOp.getTarget(), null,
                             mergeOp.getBloomFilterTarget(), true);
                     IPageWriteCallback pageWriteCallback = pageWriteCallbackFactory.createPageWriteCallback();
                     componentBulkLoader = mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false,
@@ -418,7 +418,7 @@
     }
 
     public ILSMIndexAccessor createAccessor(AbstractLSMIndexOperationContext opCtx) {
-        return new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
+        return new LSMTreeIndexAccessor(getHarness(), opCtx, getCursorFactory());
     }
 
     @Override
@@ -483,8 +483,28 @@
             returnDeletedTuples = true;
         }
         IIndexCursorStats stats = new IndexCursorStats();
-        LSMBTreeRangeSearchCursor cursor = new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples, stats);
+        LSMBTreeRangeSearchCursor cursor = createCursor(opCtx, returnDeletedTuples, stats);
         return new LSMBTreeMergeOperation(accessor, cursor, stats, mergeFileRefs.getInsertIndexFileReference(),
                 mergeFileRefs.getBloomFilterFileReference(), callback, getIndexIdentifier());
     }
+
+    public LSMBTreeBatchPointSearchCursor createBatchPointSearchCursor(ILSMIndexOperationContext opCtx) {
+        return new LSMBTreeBatchPointSearchCursor(opCtx);
+    }
+
+    protected LSMBTreeRangeSearchCursor createCursor(AbstractLSMIndexOperationContext opCtx,
+            boolean returnDeletedTuples, IIndexCursorStats stats) {
+        return new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples, stats);
+    }
+
+    /**
+     * @return Merge component factory (could be different from {@link #componentFactory}
+     */
+    protected ILSMDiskComponentFactory getMergeComponentFactory() {
+        return componentFactory;
+    }
+
+    protected ICursorFactory getCursorFactory() {
+        return cursorFactory;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java
index 8ab6fb1..73d06d2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java
@@ -21,14 +21,15 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
 import org.apache.hyracks.storage.am.btree.impls.BatchPredicate;
-import org.apache.hyracks.storage.am.btree.impls.DiskBTreePointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 
 /**
  * This cursor performs point searches for each batch of search keys.
  * Assumption: the search keys must be sorted into the increasing order.
- *
  */
 public class LSMBTreeBatchPointSearchCursor extends LSMBTreePointSearchCursor {
 
@@ -42,7 +43,9 @@
         while (!foundTuple && batchPred.hasNext()) {
             batchPred.next();
             if (foundIn >= 0) {
-                btreeCursors[foundIn].close();
+                if (operationalComponents.get(foundIn).getType() == LSMComponentType.MEMORY) {
+                    btreeCursors[foundIn].close();
+                }
                 foundIn = -1;
             }
             foundTuple = super.doHasNext();
@@ -56,6 +59,11 @@
     }
 
     @Override
+    protected ITreeIndexCursor createCursor(LSMComponentType type, BTreeAccessor btreeAccessor) {
+        return btreeAccessor.createPointCursor(false, true);
+    }
+
+    @Override
     protected boolean isSearchCandidate(int componentIndex) throws HyracksDataException {
         if (!super.isSearchCandidate(componentIndex)) {
             return false;
@@ -68,21 +76,6 @@
                 maxFileterKey, opCtx.getFilterCmp());
     }
 
-    @Override
-    protected void closeCursors() throws HyracksDataException {
-        super.closeCursors();
-        if (btreeCursors != null) {
-            // clear search states of btree cursors
-            for (int i = 0; i < numBTrees; ++i) {
-                if (btreeCursors[i] != null) {
-                    if (btreeCursors[i] instanceof DiskBTreePointSearchCursor) {
-                        ((DiskBTreePointSearchCursor) btreeCursors[i]).clearSearchState();
-                    }
-                }
-            }
-        }
-    }
-
     public int getKeyIndex() {
         return ((BatchPredicate) predicate).getKeyIndex();
     }
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 1312e30..a00e10e 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
@@ -43,7 +43,7 @@
 import org.apache.hyracks.storage.common.MultiComparator;
 import org.apache.hyracks.util.trace.ITracer;
 
-public final class LSMBTreeOpContext extends AbstractLSMIndexOperationContext {
+public class LSMBTreeOpContext extends AbstractLSMIndexOperationContext {
 
     /*
      * Finals
@@ -74,9 +74,9 @@
             IBinaryComparatorFactory[] filterCmpFactories, ITracer tracer) {
         super(index, btreeFields, filterFields, filterCmpFactories, searchCallback, modificationCallback, tracer);
         LSMBTreeMemoryComponent c = (LSMBTreeMemoryComponent) mutableComponents.get(0);
-        IBinaryComparatorFactory cmpFactories[] = c.getIndex().getComparatorFactories();
+        IBinaryComparatorFactory[] cmpFactories = c.getIndex().getComparatorFactories();
         if (cmpFactories[0] != null) {
-            this.cmp = MultiComparator.create(c.getIndex().getComparatorFactories());
+            this.cmp = createMultiComparator(c.getIndex().getComparatorFactories());
         } else {
             this.cmp = null;
         }
@@ -112,6 +112,10 @@
         insertSearchCursor = new LSMBTreePointSearchCursor(this);
     }
 
+    protected MultiComparator createMultiComparator(IBinaryComparatorFactory[] cmpFactories) {
+        return MultiComparator.create(cmpFactories);
+    }
+
     @Override
     public void setOperation(IndexOperation newOp) {
         reset();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
index d4903d9..9740bc4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
@@ -199,7 +199,8 @@
         for (int i = 0; i < numBTrees; i++) {
             ILSMComponent component = operationalComponents.get(i);
             BTree btree = (BTree) component.getIndex();
-            if (component.getType() == LSMComponentType.MEMORY) {
+            LSMComponentType type = component.getType();
+            if (type == LSMComponentType.MEMORY) {
                 includeMutableComponent = true;
                 if (bloomFilters[i] != null) {
                     destroyAndNullifyCursorAtIndex(i);
@@ -212,8 +213,8 @@
             }
 
             if (btreeAccessors[i] == null) {
-                btreeAccessors[i] = btree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-                btreeCursors[i] = btreeAccessors[i].createPointCursor(false, false);
+                btreeAccessors[i] = createAccessor(type, btree, i);
+                btreeCursors[i] = createCursor(type, btreeAccessors[i]);
             } else {
                 // re-use
                 btreeAccessors[i].reset(btree, NoOpIndexAccessParameters.INSTANCE);
@@ -225,6 +226,14 @@
         hashComputed = false;
     }
 
+    protected BTreeAccessor createAccessor(LSMComponentType type, BTree btree, int i) throws HyracksDataException {
+        return btree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+    }
+
+    protected ITreeIndexCursor createCursor(LSMComponentType type, BTreeAccessor btreeAccessor) {
+        return btreeAccessor.createPointCursor(false, false);
+    }
+
     private void destroyAndNullifyCursorAtIndex(int i) throws HyracksDataException {
         // component at location i was a disk component before, and is now a memory component, or vise versa
         bloomFilters[i] = null;
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 2c5fb50..968416c 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
@@ -49,7 +49,7 @@
     private final RangePredicate reusablePred;
     private ISearchOperationCallback searchCallback;
     private BTreeAccessor[] btreeAccessors;
-    private boolean[] isMemoryComponent;
+    protected boolean[] isMemoryComponent;
     private ArrayTupleBuilder tupleBuilder;
     private boolean canCallProceed = true;
     private boolean resultOfSearchCallbackProceed = false;
@@ -149,6 +149,7 @@
                                 // There are no more elements in the memory component.. can safely skip locking for the
                                 // remaining operations
                                 includeMutableComponent = false;
+                                excludeMemoryComponent();
                             }
                         }
                     } else {
@@ -180,6 +181,7 @@
                         // the tree of head tuple
                         // the head element of PQ is useless now
                         PriorityQueueElement e = outputPriorityQueue.poll();
+                        markAsDeleted(e);
                         pushIntoQueueFromCursorAndReplaceThisElement(e);
                     } else {
                         // If the previous tuple and the head tuple are different
@@ -200,6 +202,14 @@
 
     }
 
+    protected void excludeMemoryComponent() {
+        //NoOp
+    }
+
+    protected void markAsDeleted(PriorityQueueElement e) throws HyracksDataException {
+        //NoOp
+    }
+
     private void pushOutputElementIntoQueueIfNeeded() throws HyracksDataException {
         if (needPushElementIntoQueue) {
             pushIntoQueueFromCursorAndReplaceThisElement(outputElement);
@@ -250,7 +260,7 @@
         }
     }
 
-    private int replaceFrom() throws HyracksDataException {
+    protected int replaceFrom() throws HyracksDataException {
         int replaceFrom = -1;
         if (!switchPossible) {
             return replaceFrom;
@@ -386,20 +396,21 @@
         }
         for (int i = 0; i < numBTrees; i++) {
             ILSMComponent component = operationalComponents.get(i);
+            LSMComponentType type = component.getType();
             BTree btree;
             if (component.getType() == LSMComponentType.MEMORY) {
                 includeMutableComponent = true;
             }
             btree = (BTree) component.getIndex();
             if (btreeAccessors[i] == null || destroyIncompatible(component, i)) {
-                btreeAccessors[i] = btree.createAccessor(iap);
-                rangeCursors[i] = btreeAccessors[i].createSearchCursor(false);
+                btreeAccessors[i] = createAccessor(type, btree, i);
+                rangeCursors[i] = createCursor(type, btreeAccessors[i]);
             } else {
                 // re-use
                 btreeAccessors[i].reset(btree, iap);
                 rangeCursors[i].close();
             }
-            isMemoryComponent[i] = component.getType() == LSMComponentType.MEMORY;
+            isMemoryComponent[i] = type == LSMComponentType.MEMORY;
         }
         IndexCursorUtils.open(btreeAccessors, rangeCursors, searchPred);
         try {
@@ -433,4 +444,12 @@
         return resultOfSearchCallbackProceed;
     }
 
+    protected BTreeAccessor createAccessor(LSMComponentType type, BTree btree, int index) throws HyracksDataException {
+        return btree.createAccessor(iap);
+    }
+
+    protected IIndexCursor createCursor(LSMComponentType type, BTreeAccessor accessor) {
+        return accessor.createSearchCursor(false);
+    }
+
 }
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 efacad1..aa72267 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
@@ -46,6 +46,13 @@
         scanCursor = new LSMBTreeDiskComponentScanCursor(opCtx);
     }
 
+    protected LSMBTreeSearchCursor(LSMBTreePointSearchCursor pointCursor, LSMBTreeRangeSearchCursor rangeCursor,
+            LSMBTreeDiskComponentScanCursor scanCursor) {
+        this.pointCursor = pointCursor;
+        this.rangeCursor = rangeCursor;
+        this.scanCursor = scanCursor;
+    }
+
     @Override
     public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         LSMBTreeCursorInitialState lsmInitialState = (LSMBTreeCursorInitialState) initialState;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
index 13a0e27..acb84e1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
@@ -20,7 +20,7 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex.AbstractTreeIndexBulkLoader;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndexBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleWriter;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
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 27875c0..2c97221 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
@@ -236,7 +236,7 @@
                     } else {
                         // If the previous tuple and the head tuple are different
                         // the info of previous tuple is useless
-                        if (needPushElementIntoQueue == true) {
+                        if (needPushElementIntoQueue) {
                             pushIntoQueueFromCursorAndReplaceThisElement(outputElement);
                             needPushElementIntoQueue = false;
                         }
@@ -304,6 +304,10 @@
         public MultiComparator getMultiComparator() {
             return cmp;
         }
+
+        public void setMultiComparator(MultiComparator cmp) {
+            this.cmp = cmp;
+        }
     }
 
     protected void setPriorityQueueComparator() {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
index b7eb115..11385de 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.common.api.ILSMIndexCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
@@ -50,7 +51,7 @@
 
     protected boolean open;
     protected RTreeSearchCursor[] rtreeCursors;
-    protected BTreeRangeSearchCursor[] btreeCursors;
+    protected ITreeIndexCursor[] btreeCursors;
     protected RTreeAccessor[] rtreeAccessors;
     protected BTreeAccessor[] btreeAccessors;
     protected BloomFilter[] bloomFilters;
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 8e5cb35..729ca74 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
@@ -142,7 +142,8 @@
                 bTreeTupleSorter = new TreeTupleSorter(flushingComponent.getBuddyIndex().getFileId(), linearizerArray,
                         btreeLeafFrameFactory.createFrame(), btreeLeafFrameFactory.createFrame(),
                         flushingComponent.getBuddyIndex().getBufferCache(), comparatorFields);
-                BTreeRangeSearchCursor btreeScanCursor = memBTreeAccessor.createSearchCursor(false);
+                BTreeRangeSearchCursor btreeScanCursor =
+                        (BTreeRangeSearchCursor) memBTreeAccessor.createSearchCursor(false);
                 try {
                     isEmpty = true;
                     memBTreeAccessor.search(btreeScanCursor, btreeNullPredicate);
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 a8a4252..e1c6f5b 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
@@ -28,6 +28,7 @@
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
 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.impls.DefaultTupleProjectorFactory;
 import org.apache.hyracks.storage.am.rtree.impls.SearchPredicate;
 import org.apache.hyracks.storage.am.rtree.util.RTreeUtils;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
@@ -59,7 +60,7 @@
         super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
                 nonFilterWriterFactory, null, -1, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
-                searchCallbackProceedResultTrueValue);
+                searchCallbackProceedResultTrueValue, DefaultTupleProjectorFactory.INSTANCE);
         if (keyFields != null && keyFields.length > 0) {
             searchKey = new PermutingFrameTupleReference();
             searchKey.setFieldPermutation(keyFields);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
index 7e8f249..d85200f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
@@ -41,6 +41,7 @@
 import org.apache.hyracks.storage.am.common.frames.AbstractSlotManager;
 import org.apache.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
 import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndexBulkLoader;
 import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
 import org.apache.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
@@ -902,7 +903,7 @@
         return new RTreeBulkLoader(fillFactor, callback);
     }
 
-    public class RTreeBulkLoader extends AbstractTreeIndex.AbstractTreeIndexBulkLoader {
+    public class RTreeBulkLoader extends AbstractTreeIndexBulkLoader {
         ITreeIndexFrame lowerFrame, prevInteriorFrame;
         RTreeTypeAwareTupleWriter interiorFrameTupleWriter =
                 ((RTreeTypeAwareTupleWriter) interiorFrame.getTupleWriter());
@@ -911,7 +912,7 @@
         List<Integer> prevNodeFrontierPages = new ArrayList<>();
 
         public RTreeBulkLoader(float fillFactor, IPageWriteCallback callback) throws HyracksDataException {
-            super(fillFactor, callback);
+            super(fillFactor, callback, RTree.this);
             prevInteriorFrame = interiorFrameFactory.createFrame();
         }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexAccessParameters.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexAccessParameters.java
index a3745d0..8f553d9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexAccessParameters.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexAccessParameters.java
@@ -40,4 +40,14 @@
      * Gets additional parameters.
      */
     Map<String, Object> getParameters();
+
+    /**
+     * Gets a parameter.
+     *
+     * @param key   of a parameter
+     * @param clazz used to explicitly cast the requested parameter to the required type
+     * @param <T>   the required type
+     * @return the requested parameter
+     */
+    <T> T getParameter(String key, Class<T> clazz);
 }
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 f1fe86f..5578d27 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
@@ -552,6 +552,10 @@
         BufferedFileHandle fInfo = getFileHandle(cPage);
         cPage.buffer.clear();
         fInfo.read(cPage);
+        final IThreadStats threadStats = statsSubscribers.get(Thread.currentThread());
+        if (threadStats != null) {
+            threadStats.coldRead();
+        }
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
index 6095b26..8ca1a82 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
@@ -16,9 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.hyracks.storage.common.projection;
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+import java.io.DataOutput;
+import java.io.IOException;
 
-select value v from range(1,2) v where v > ?;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public interface ITupleProjector {
+    void project(ITupleReference tuple, DataOutput dos, ArrayTupleBuilder tb) throws IOException;
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
similarity index 61%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
index 6095b26..ff9ecf9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
@@ -16,9 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.hyracks.storage.common.projection;
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+import java.io.Serializable;
 
-select value v from range(1,2) v where v > ?;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Tuple projector allows the data source to project the values needed before it pushed up
+ * to the upper operator.
+ */
+public interface ITupleProjectorFactory extends Serializable {
+    ITupleProjector createTupleProjector(IHyracksTaskContext context) throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
index dcb85f6..c73ebee 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
@@ -49,15 +49,17 @@
 public class TestTaskContext implements IHyracksTaskContext {
     private final TestJobletContext jobletContext;
     private final TaskAttemptId taskId;
+    private int partitionCount;
     private WorkspaceFileFactory fileFactory;
     private Map<Object, IStateObject> stateObjectMap = new HashMap<>();
     private Object sharedObject;
     private final IStatsCollector statsCollector = new StatsCollector();
     private final ThreadStats threadStats = new ThreadStats();
 
-    public TestTaskContext(TestJobletContext jobletContext, TaskAttemptId taskId) {
+    public TestTaskContext(TestJobletContext jobletContext, TaskAttemptId taskId, int partitionCount) {
         this.jobletContext = jobletContext;
         this.taskId = taskId;
+        this.partitionCount = partitionCount;
         fileFactory = new WorkspaceFileFactory(this, getIoManager());
     }
 
@@ -129,6 +131,11 @@
     }
 
     @Override
+    public int getPartitionCount() {
+        return partitionCount;
+    }
+
+    @Override
     public synchronized void setStateObject(IStateObject taskState) {
         stateObjectMap.put(taskState.getId(), taskState);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
index 3f78234..2348a1a 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
@@ -38,7 +38,7 @@
 import org.apache.hyracks.api.dataflow.TaskId;
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
 import org.apache.hyracks.api.io.IODeviceHandle;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.util.CleanupUtils;
@@ -52,22 +52,7 @@
 public class TestUtils {
 
     private static final int DEFAULT_FRAME_SIZE = 32768;
-    public static final IWarningCollector NOOP_WARNING_COLLECTOR = new IWarningCollector() {
-        @Override
-        public void warn(Warning warning) {
-            // no-op
-        }
-
-        @Override
-        public boolean shouldWarn() {
-            return false;
-        }
-
-        @Override
-        public long getTotalWarningsCount() {
-            return 0;
-        }
-    };
+    public static final IWarningCollector NOOP_WARNING_COLLECTOR = NoOpWarningCollector.INSTANCE;
 
     public static IHyracksTaskContext createHyracksTask() {
         return create(DEFAULT_FRAME_SIZE);
@@ -91,7 +76,7 @@
             INCServiceContext serviceCtx = new TestNCServiceContext(ioManager, null);
             TestJobletContext jobletCtx = new TestJobletContext(frameSize, serviceCtx, new JobId(0));
             TaskAttemptId tid = new TaskAttemptId(new TaskId(new ActivityId(new OperatorDescriptorId(0), 0), 0), 0);
-            IHyracksTaskContext taskCtx = new TestTaskContext(jobletCtx, tid);
+            IHyracksTaskContext taskCtx = new TestTaskContext(jobletCtx, tid, 1);
             return taskCtx;
         } catch (HyracksException e) {
             throw new RuntimeException(e);
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/TestLsmBtreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
index 8e2d6dd..c7d3dff 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
@@ -56,19 +56,21 @@
             ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
-            ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter) {
+            ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter,
+            boolean isSecondaryNoIncrementalMaintenance) {
         super(typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, path,
                 storageManager, mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories,
                 btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
                 metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, durable,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null);
+                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null,
+                isSecondaryNoIncrementalMaintenance);
     }
 
     protected TestLsmBtreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
-            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter)
-            throws HyracksDataException {
+            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
+            boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
         super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter);
+                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
     }
 
     @Override
@@ -107,7 +109,8 @@
         final boolean isPrimary = json.get("isPrimary").asBoolean();
         final boolean hasBloomFilter = json.get("hasBloomFilter").asBoolean();
         final int[] btreeFields = OBJECT_MAPPER.convertValue(json.get("btreeFields"), int[].class);
+        boolean isSecondaryNoIncrementalMaintenance = json.get("isSecondaryNoIncrementalMaintenance").asBoolean();
         return new TestLsmBtreeLocalResource(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
-                isPrimary, btreeFields, hasBloomFilter);
+                isPrimary, btreeFields, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
     }
 }
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/TestLsmBtreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
index 7d9b24a..48adf91 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
@@ -46,12 +46,14 @@
             IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
             Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
-            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter) {
+            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
+            boolean isSecondaryNoIncrementalMaintenance) {
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
                 vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable,
                 bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null);
+                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null,
+                isSecondaryNoIncrementalMaintenance);
     }
 
     @Override
@@ -60,6 +62,6 @@
                 bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
                 mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                vbcProvider, ioSchedulerProvider, durable, hasBloomFilter);
+                vbcProvider, ioSchedulerProvider, durable, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
     }
 }
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 aa9cd5a..144f1ad 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
@@ -740,6 +740,11 @@
         }
 
         @Override
+        public int getPartitionCount() {
+            return 1;
+        }
+
+        @Override
         public ICounterContext getCounterContext() {
             return null;
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IThreadStats.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IThreadStats.java
index dc63ac4..e782a69 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IThreadStats.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IThreadStats.java
@@ -31,4 +31,16 @@
      * @return the pinned pages count
      */
     long getPinnedPagesCount();
+
+    /**
+     * Indicates that this thread caused a cold read from disk
+     */
+    void coldRead();
+
+    /**
+     * Gets the count of pages read in from disk
+     *
+     * @return the cold read count
+     */
+    long getColdReadCount();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThreadStats.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThreadStats.java
index c79eefc..9af3fb1 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThreadStats.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThreadStats.java
@@ -26,6 +26,7 @@
 public class ThreadStats implements IThreadStats {
 
     private AtomicLong pinnedPagesCount = new AtomicLong();
+    private AtomicLong coldReadCount = new AtomicLong();
 
     @Override
     public void pagePinned() {
@@ -36,4 +37,14 @@
     public long getPinnedPagesCount() {
         return pinnedPagesCount.get();
     }
+
+    @Override
+    public long getColdReadCount() {
+        return coldReadCount.get();
+    }
+
+    @Override
+    public void coldRead() {
+        coldReadCount.incrementAndGet();
+    }
 }
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 3eb8687..c0475b1 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
@@ -244,12 +244,13 @@
      * consistent with the comparison result.
      */
     public static int normalize(byte[] bytes, int start) {
-        int len = getUTFLength(bytes, start);
         long nk = 0;
+        int len = getUTFLength(bytes, start);
         int offset = start + getNumBytesToStoreLength(len);
+        int end = offset + len;
         for (int i = 0; i < 2; ++i) {
             nk <<= 16;
-            if (i < len) {
+            if (offset < end) {
                 nk += (charAt(bytes, offset)) & 0xffff;
                 offset += charSize(bytes, offset);
             }
@@ -498,19 +499,15 @@
      * are exactly the same as for the <code>readUTF</code>
      * method of <code>DataInput</code>.
      *
-     * @param in
-     *            a data input stream.
+     * @param in a data input stream.
      * @return a Unicode string.
-     * @throws EOFException
-     *             if the input stream reaches the end
-     *             before all the bytes.
-     * @throws IOException
-     *             the stream has been closed and the contained
-     *             input stream does not support reading after close, or
-     *             another I/O error occurs.
-     * @throws UTFDataFormatException
-     *             if the bytes do not represent a
-     *             valid modified UTF-8 encoding of a Unicode string.
+     * @throws EOFException           if the input stream reaches the end
+     *                                before all the bytes.
+     * @throws IOException            the stream has been closed and the contained
+     *                                input stream does not support reading after close, or
+     *                                another I/O error occurs.
+     * @throws UTFDataFormatException if the bytes do not represent a
+     *                                valid modified UTF-8 encoding of a Unicode string.
      * @see java.io.DataInputStream#readUnsignedShort()
      */
     public static String readUTF8(DataInput in) throws IOException {
@@ -602,10 +599,8 @@
     /**
      * Write a UTF8 String <code>str</code> into the DataOutput <code>out</code>
      *
-     * @param str,
-     *            a Unicode string;
-     * @param out,
-     *            a Data output stream.
+     * @param str, a Unicode string;
+     * @param out, a Data output stream.
      * @throws IOException
      */
     public static void writeUTF8(CharSequence str, DataOutput out) throws IOException {
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringSample.java b/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringSample.java
index b114351..eb3a5b6 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringSample.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringSample.java
@@ -35,7 +35,8 @@
 
     public static final String STRING_LEN_3 = "xyz";
     public static final String STRING_UTF8_3 = "锟斤拷";
-    public static final String STRING_UTF8_MIX = "\uD841\uDF0E\uD841\uDF31锟X斤Y拷Zà"; // one, two, three, and four bytes
+    // one, two, three, and four bytes
+    public static final String STRING_UTF8_MIX = "\uD841\uDF0E\uD841\uDF31锟X斤Y拷Zà";
     public static final String STRING_UTF8_MIX_LOWERCASE = "\uD841\uDF0E\uD841\uDF31锟x斤y拷zà";
     public static final String STRING_NEEDS_2_JAVA_CHARS_1 = "\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89";
     public static final String STRING_NEEDS_2_JAVA_CHARS_2 = "😢😢💉💉";
@@ -44,6 +45,8 @@
     public static final String STRING_EMOJI_FAMILY_OF_2 = "\uD83D\uDC68\u200D\uD83D\uDC66";
     public static final String EMOJI_BASKETBALL = "\uD83C\uDFC0";
 
+    public static final String THREE_BYTES_UTF8_CHAR = "ह";
+
     public static final String STRING_LEN_127 = generateStringRepeatBy(ONE_ASCII_CHAR, 127);
     public static final String STRING_LEN_128 = generateStringRepeatBy(ONE_ASCII_CHAR, 128);
 
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringUtilTest.java b/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringUtilTest.java
index c7468d2..4eb1fc3 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringUtilTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/string/UTF8StringUtilTest.java
@@ -25,6 +25,7 @@
 import static org.apache.hyracks.util.string.UTF8StringSample.STRING_UTF8_3;
 import static org.apache.hyracks.util.string.UTF8StringSample.STRING_UTF8_MIX;
 import static org.apache.hyracks.util.string.UTF8StringSample.STRING_UTF8_MIX_LOWERCASE;
+import static org.apache.hyracks.util.string.UTF8StringSample.THREE_BYTES_UTF8_CHAR;
 import static org.apache.hyracks.util.string.UTF8StringUtil.charAt;
 import static org.apache.hyracks.util.string.UTF8StringUtil.charSize;
 import static org.apache.hyracks.util.string.UTF8StringUtil.compareTo;
@@ -77,13 +78,14 @@
     }
 
     @Test
-    public void testCompareToAndNormolize() throws Exception {
+    public void testCompareToAndNormalize() throws Exception {
         testCompare(STRING_UTF8_MIX, STRING_UTF8_MIX, OPTION.STANDARD);
         testCompare(STRING_UTF8_3, STRING_UTF8_MIX, OPTION.STANDARD);
         testCompare(STRING_LEN_MEDIUM, STRING_UTF8_MIX, OPTION.STANDARD);
+        testCompare(THREE_BYTES_UTF8_CHAR, THREE_BYTES_UTF8_CHAR, OPTION.STANDARD);
     }
 
-    public boolean isSameSign(int r1, int r2) {
+    private static boolean isSameSign(int r1, int r2) {
         if (r1 > 0) {
             return r2 > 0;
         }
@@ -99,7 +101,7 @@
         LOWERCASE
     }
 
-    public void testCompare(String str1, String str2, OPTION option) throws IOException {
+    private static void testCompare(String str1, String str2, OPTION option) {
         byte[] buffer1 = writeStringToBytes(str1);
         byte[] buffer2 = writeStringToBytes(str2);
 
@@ -117,7 +119,6 @@
                 assertEquals(str1.compareToIgnoreCase(str2), lowerCaseCompareTo(buffer1, 0, buffer2, 0));
                 break;
         }
-
     }
 
     @Test
diff --git a/hyracks-fullstack/hyracks/pom.xml b/hyracks-fullstack/hyracks/pom.xml
index 443e8c0..bfd71e0 100644
--- a/hyracks-fullstack/hyracks/pom.xml
+++ b/hyracks-fullstack/hyracks/pom.xml
@@ -88,6 +88,7 @@
     <module>hyracks-storage-am-lsm-invertedindex</module>
     <module>hyracks-storage-am-lsm-common</module>
     <module>hyracks-storage-am-lsm-btree</module>
+    <module>hyracks-storage-am-lsm-btree-column</module>
     <module>hyracks-storage-am-lsm-rtree</module>
     <module>hyracks-storage-am-rtree</module>
     <module>hyracks-test-support</module>