Null merge branch 'gerrit/neo' into 'gerrit/trinity'

Change-Id: I1a239eba807beed823460b491a43991226f5ff09
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveEvent.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveEvent.java
index d6d9b96..279bba1 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveEvent.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveEvent.java
@@ -20,6 +20,7 @@
 
 import java.util.Objects;
 
+import org.apache.asterix.active.message.ActivePartitionMessage;
 import org.apache.hyracks.api.job.JobId;
 
 public class ActiveEvent {
@@ -65,7 +66,12 @@
 
     @Override
     public String toString() {
-        return "JobId:" + jobId + "," + "EntityId:" + entityId + ", " + "Kind" + eventKind;
+        String kindDesc = "";
+        if (eventObject instanceof ActivePartitionMessage) {
+            ActivePartitionMessage partitionEvent = (ActivePartitionMessage) eventObject;
+            kindDesc = '-' + String.valueOf(partitionEvent.getEvent()) + '(' + partitionEvent.getDesc() + ')';
+        }
+        return jobId + ", " + "EntityId:" + entityId + ", " + "Kind:" + eventKind + kindDesc;
     }
 
     @Override
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
index b99e4f2..636279c 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
@@ -103,8 +103,8 @@
         return ActiveManager.class.getSimpleName() + "[" + nodeId + "]";
     }
 
-    public void submit(ActiveManagerMessage message) throws HyracksDataException {
-        LOGGER.debug("Message of type {} received in {}", message.getKind(), nodeId);
+    public void handle(ActiveManagerMessage message) throws HyracksDataException {
+        LOGGER.debug("NC handling {}({})({})", message.getKind(), message.getRuntimeId(), message.getDesc());
         switch (message.getKind()) {
             case STOP_ACTIVITY:
                 stopRuntime(message);
@@ -125,7 +125,7 @@
             ActiveRuntimeId runtimeId = message.getRuntimeId();
             IActiveRuntime runtime = runtimes.get(runtimeId);
             if (runtime == null) {
-                LOGGER.warn("Request for a runtime {} that is not registered {}", runtimeId, message);
+                LOGGER.warn("not handling {} for a runtime {} that is not registered", message, runtimeId);
                 return;
             }
             runtime.handleGenericEvent(message);
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveSourceOperatorNodePushable.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveSourceOperatorNodePushable.java
index 453ffa0..9123503 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveSourceOperatorNodePushable.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveSourceOperatorNodePushable.java
@@ -27,7 +27,6 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
-import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -98,14 +97,14 @@
         try {
             // notify cc that runtime has been registered
             ctx.sendApplicationMessageToCC(new ActivePartitionMessage(runtimeId, ctx.getJobletContext().getJobId(),
-                    Event.RUNTIME_REGISTERED, null), null);
+                    Event.RUNTIME_REGISTERED, null, ""), null);
             start();
         } catch (InterruptedException e) {
-            LOGGER.log(Level.INFO, "initialize() interrupted on ActiveSourceOperatorNodePushable", e);
+            LOGGER.info("ingestion op interrupted", e);
             Thread.currentThread().interrupt();
             throw HyracksDataException.create(e);
         } catch (Exception e) {
-            LOGGER.log(Level.INFO, "initialize() failed on ActiveSourceOperatorNodePushable", e);
+            logIngestionFailure(e);
             throw HyracksDataException.create(e);
         } finally {
             synchronized (this) {
@@ -121,12 +120,12 @@
         activeManager.deregisterRuntime(runtimeId);
         try {
             ctx.sendApplicationMessageToCC(new ActivePartitionMessage(runtimeId, ctx.getJobletContext().getJobId(),
-                    Event.RUNTIME_DEREGISTERED, null), null);
+                    Event.RUNTIME_DEREGISTERED, null, ""), null);
         } catch (Exception e) {
-            LOGGER.log(Level.INFO, "deinitialize() failed on ActiveSourceOperatorNodePushable", e);
+            LOGGER.info("ingestion op stopped w/ failure", e);
             throw HyracksDataException.create(e);
         } finally {
-            LOGGER.log(Level.INFO, "deinitialize() returning on ActiveSourceOperatorNodePushable");
+            LOGGER.info("ingestion op stopped");
         }
     }
 
@@ -134,4 +133,12 @@
     public final IFrameWriter getInputFrameWriter(int index) {
         return null;
     }
+
+    private void logIngestionFailure(Exception e) {
+        if (e.getCause() instanceof InterruptedException) {
+            LOGGER.info("ingestion op interrupted", e);
+        } else {
+            LOGGER.info("ingestion op failed", e);
+        }
+    }
 }
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveManagerMessage.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveManagerMessage.java
index bad3f79..692bbb9 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveManagerMessage.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveManagerMessage.java
@@ -34,15 +34,17 @@
         GENERIC_EVENT
     }
 
-    private static final long serialVersionUID = 2L;
+    private static final long serialVersionUID = 3L;
     private final Kind kind;
     private final ActiveRuntimeId runtimeId;
     private final Serializable payload;
+    private final String desc;
 
-    public ActiveManagerMessage(Kind kind, ActiveRuntimeId runtimeId, Serializable payload) {
+    public ActiveManagerMessage(Kind kind, ActiveRuntimeId runtimeId, Serializable payload, String desc) {
         this.kind = kind;
         this.runtimeId = runtimeId;
         this.payload = payload;
+        this.desc = desc;
     }
 
     public Serializable getPayload() {
@@ -57,13 +59,18 @@
         return kind;
     }
 
+    public String getDesc() {
+        return desc;
+    }
+
     @Override
     public void handle(INcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
-        ((ActiveManager) appCtx.getActiveManager()).submit(this);
+        ((ActiveManager) appCtx.getActiveManager()).handle(this);
     }
 
     @Override
     public String toString() {
-        return getClass().getSimpleName() + "{" + "kind=" + kind + ", runtimeId=" + runtimeId + '}';
+        return getClass().getSimpleName() + "{kind=" + kind + ", runtimeId=" + runtimeId
+                + (desc != null && !desc.isEmpty() ? ", desc=" + desc : "") + '}';
     }
 }
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActivePartitionMessage.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActivePartitionMessage.java
index cb9c61b..2955271 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActivePartitionMessage.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActivePartitionMessage.java
@@ -29,23 +29,27 @@
 import org.apache.hyracks.api.job.JobId;
 
 public class ActivePartitionMessage implements ICcAddressedMessage {
+
     public enum Event {
         RUNTIME_REGISTERED,
         RUNTIME_DEREGISTERED,
         GENERIC_EVENT
     }
 
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 2L;
     private final ActiveRuntimeId activeRuntimeId;
     private final JobId jobId;
     private final Serializable payload;
+    private final String desc;
     private final Event event;
 
-    public ActivePartitionMessage(ActiveRuntimeId activeRuntimeId, JobId jobId, Event event, Serializable payload) {
+    public ActivePartitionMessage(ActiveRuntimeId activeRuntimeId, JobId jobId, Event event, Serializable payload,
+            String desc) {
         this.activeRuntimeId = activeRuntimeId;
         this.jobId = jobId;
         this.event = event;
         this.payload = payload;
+        this.desc = desc;
     }
 
     public ActiveRuntimeId getActiveRuntimeId() {
@@ -64,6 +68,10 @@
         return event;
     }
 
+    public String getDesc() {
+        return desc;
+    }
+
     @Override
     public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
         IActiveNotificationHandler activeListener = (IActiveNotificationHandler) appCtx.getActiveNotificationHandler();
@@ -72,7 +80,7 @@
 
     @Override
     public String toString() {
-        return activeRuntimeId + ":" + ActivePartitionMessage.class.getSimpleName() + '-' + event;
+        return activeRuntimeId + ":" + ActivePartitionMessage.class.getSimpleName() + '-' + event + '(' + desc + ')';
     }
 
     @Override
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveStatsRequestMessage.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveStatsRequestMessage.java
index 94668a0..2e5a571 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveStatsRequestMessage.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveStatsRequestMessage.java
@@ -25,7 +25,7 @@
     private final long reqId;
 
     public ActiveStatsRequestMessage(ActiveRuntimeId runtimeId, long reqId) {
-        super(Kind.REQUEST_STATS, runtimeId, null);
+        super(Kind.REQUEST_STATS, runtimeId, null, "");
         this.reqId = reqId;
     }
 
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/SqlppCompilationProvider.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
index 4ad888c..66da5e2 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
@@ -37,6 +37,7 @@
 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.cbo.JoinEnum;
 import org.apache.asterix.optimizer.rules.util.EquivalenceClassUtils;
 import org.apache.asterix.translator.SqlppExpressionToPlanTranslator;
 import org.apache.asterix.translator.SqlppExpressionToPlanTranslatorFactory;
@@ -92,6 +93,7 @@
                 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));
+                EquivalenceClassUtils.REWRITE_INTERNAL_QUERYUID_PK, SqlppQueryRewriter.SQL_COMPAT_OPTION,
+                JoinEnum.CBO_FULL_ENUM_LEVEL_KEY, JoinEnum.CBO_CP_ENUM_KEY));
     }
 }
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/AsterixOptimizationContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java
index 05e7f7b..733da2d 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
@@ -24,6 +24,7 @@
 import java.util.function.Predicate;
 
 import org.apache.asterix.metadata.declared.DataSource;
+import org.apache.asterix.metadata.declared.FunctionDataSource;
 import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IConflictingTypeResolver;
@@ -83,10 +84,10 @@
         }
         for (Int2ObjectMap.Entry<Set<DataSource>> me : dataSourceMap.int2ObjectEntrySet()) {
             int dataSourceType = me.getIntKey();
-            if (dataSourceType != DataSource.Type.INTERNAL_DATASET) {
+            if (dataSourceType != DataSource.Type.INTERNAL_DATASET && dataSourceType != DataSource.Type.FUNCTION) {
                 return false;
             }
-            Predicate<DataSource> dataSourceTest = AsterixOptimizationContext::isMetadata;
+            Predicate<DataSource> dataSourceTest = AsterixOptimizationContext::skipJobCapacityAssignment;
             if (!me.getValue().stream().allMatch(dataSourceTest)) {
                 return false;
             }
@@ -94,7 +95,11 @@
         return true;
     }
 
-    private static boolean isMetadata(DataSource ds) {
-        return MetadataConstants.METADATA_DATAVERSE_NAME.equals(ds.getId().getDataverseName());
+    private static boolean skipJobCapacityAssignment(DataSource ds) {
+        return MetadataConstants.METADATA_DATAVERSE_NAME.equals(ds.getId().getDataverseName()) || isSkipping(ds);
+    }
+
+    private static boolean isSkipping(DataSource ds) {
+        return (ds instanceof FunctionDataSource) && ((FunctionDataSource) ds).skipJobCapacityAssignment();
     }
 }
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 1c3ef03..3cea23f 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
@@ -112,6 +112,7 @@
 import org.apache.hyracks.algebricks.rewriter.rules.BreakSelectIntoConjunctsRule;
 import org.apache.hyracks.algebricks.rewriter.rules.ComplexUnnestToProductRule;
 import org.apache.hyracks.algebricks.rewriter.rules.ConsolidateAssignsRule;
+import org.apache.hyracks.algebricks.rewriter.rules.ConsolidateLeftOuterJoinSelectsRule;
 import org.apache.hyracks.algebricks.rewriter.rules.ConsolidateSelectsRule;
 import org.apache.hyracks.algebricks.rewriter.rules.CopyLimitDownRule;
 import org.apache.hyracks.algebricks.rewriter.rules.EliminateGroupByEmptyKeyRule;
@@ -339,6 +340,8 @@
         planCleanupRules.add(new RemoveRedundantVariablesInUnionRule()); // relies on RemoveUnusedAssignAndAggregateRule
         planCleanupRules.add(new PushProjectDownRule());
         planCleanupRules.add(new PushSelectDownRule());
+        // Must run after PushSelectDownRule
+        planCleanupRules.add(new ConsolidateLeftOuterJoinSelectsRule());
         planCleanupRules.add(new SetClosedRecordConstructorsRule());
         planCleanupRules.add(new IntroduceDynamicTypeCastRule());
         planCleanupRules.add(new IntroduceDynamicTypeCastForExternalFunctionRule());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CancelUnnestWithNestedListifyRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CancelUnnestWithNestedListifyRule.java
index 069ba49..4974fed 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CancelUnnestWithNestedListifyRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CancelUnnestWithNestedListifyRule.java
@@ -219,6 +219,10 @@
         if (arg0.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
             return false;
         }
+        AbstractLogicalOperator aggChild = (AbstractLogicalOperator) agg.getInputs().get(0).getValue();
+        if (aggChild.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
+            return false;
+        }
         LogicalVariable paramVar = ((VariableReferenceExpression) arg0).getVariableReference();
 
         ArrayList<LogicalVariable> assgnVars = new ArrayList<LogicalVariable>(1);
@@ -246,6 +250,9 @@
             Mutable<ILogicalOperator> bottomOpRef = aggInputOpRef;
             AbstractLogicalOperator bottomOp = (AbstractLogicalOperator) bottomOpRef.getValue();
             while (bottomOp.getOperatorTag() != LogicalOperatorTag.NESTEDTUPLESOURCE) {
+                if (bottomOp.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
+                    return false;
+                }
                 bottomOpRef = bottomOp.getInputs().get(0);
                 bottomOp = (AbstractLogicalOperator) bottomOpRef.getValue();
             }
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 fe000c3..9fe1ba4 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
@@ -47,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;
@@ -393,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/SweepIllegalNonfunctionalFunctions.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
index 2509010..2a6604e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
@@ -61,6 +61,7 @@
 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.SwitchOperator;
 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;
@@ -205,6 +206,11 @@
         }
 
         @Override
+        public Void visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+            return null;
+        }
+
+        @Override
         public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
             return null;
         }
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 b66a0fd..10b763b 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
@@ -26,12 +26,15 @@
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
+import java.util.stream.IntStream;
 
 import org.apache.asterix.algebra.operators.physical.ExternalDataLookupPOperator;
 import org.apache.asterix.common.annotations.AbstractExpressionAnnotationWithIndexNames;
@@ -86,6 +89,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
@@ -94,6 +98,7 @@
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions.ComparisonKind;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractDataSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
@@ -148,6 +153,19 @@
             BuiltinFunctions.YEAR_MONTH_DURATION_DEFAULT_NULL_CONSTRUCTOR,
             BuiltinFunctions.UUID_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.BINARY_BASE64_DEFAULT_NULL_CONSTRUCTOR);
 
+    // TODO (GLENN): We can definitely expand the whitelist here...
+    private final static Map<FunctionIdentifier, Set<Integer>> INDEX_USE_ON_FUNCTION_CALL_WHITELIST;
+    private final static Set<Integer> ALL_INDEX_FUNCTION_ARGUMENTS = Collections.emptySet();
+    static {
+        INDEX_USE_ON_FUNCTION_CALL_WHITELIST = new HashMap<>();
+        INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.RECORD_ADD, Set.of(0));
+        INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.ADD_FIELDS, Set.of(0));
+        INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.RECORD_REMOVE, Set.of(0));
+        INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.RECORD_RENAME, Set.of(0));
+        INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.REMOVE_FIELDS, Set.of(0));
+        INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.RECORD_CONCAT, ALL_INDEX_FUNCTION_ARGUMENTS);
+    }
+
     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,
@@ -2989,8 +3007,26 @@
             isByName = true;
         }
         if (isFieldAccess) {
-            LogicalVariable sourceVar =
-                    ((VariableReferenceExpression) funcExpr.getArguments().get(0).getValue()).getVariableReference();
+            ILogicalExpression funcExprArg0 = funcExpr.getArguments().get(0).getValue();
+            MutableInt sourceIndicator = new MutableInt(0);
+            LogicalVariable sourceVar;
+            if (funcExprArg0.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                // This might be a field-access on an indexable-function-call (or nested indexable-function-calls).
+                List<LogicalVariable> foundDatasourceVariables = new ArrayList<>();
+                if (canUseIndexOnFunction((AbstractFunctionCallExpression) funcExprArg0, sourceIndicator,
+                        foundDatasourceVariables, optFuncExpr, op.computeInputTypeEnvironment(context), context)) {
+                    // TODO (GLENN): In the case of OBJECT_CONCAT w/ potentially multiple datasource variables, we
+                    //               will not explore each variable. This method definitely needs refactoring in the
+                    //               future to handle such a case.
+                    sourceVar = foundDatasourceVariables.get(0);
+                } else {
+                    return NO_FIELD_NAME;
+                }
+            } else if (funcExprArg0.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+                return NO_FIELD_NAME;
+            } else {
+                sourceVar = ((VariableReferenceExpression) funcExprArg0).getVariableReference();
+            }
             if (optFuncExpr != null) {
                 optFuncExpr.setLogicalExpr(funcVarIndex, parentFuncExpr);
                 optFuncExpr.addStepExpr(funcVarIndex, funcExpr);
@@ -3028,12 +3064,27 @@
             if (assignAndExpressionIndexes != null && assignAndExpressionIndexes[0] > -1) {
                 //We found the nested assign
 
-                //Recursive call on nested assign
-                Pair<List<String>, Integer> parentFieldNames =
-                        getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignAndExpressionIndexes[0],
-                                assignAndExpressionIndexes[1], funcVarIndex, parentFuncExpr, context);
+                // Is the next operator composed of functions that are not a field access? If so, do not recurse.
+                ILogicalOperator nextOp = subTree.getAssignsAndUnnests().get(assignAndExpressionIndexes[0]);
+                boolean isIndexOnFunction = false;
+                if (nextOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                    AssignOperator nextAssignOp = (AssignOperator) nextOp;
+                    ILogicalExpression leadingArgumentExpr = nextAssignOp.getExpressions().get(0).getValue();
+                    if (leadingArgumentExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                        IVariableTypeEnvironment typeEnv = nextAssignOp.computeInputTypeEnvironment(context);
+                        isIndexOnFunction = canUseIndexOnFunction((AbstractFunctionCallExpression) leadingArgumentExpr,
+                                sourceIndicator, new HashSet<>(), optFuncExpr, typeEnv, context);
+                    }
+                }
 
-                if (parentFieldNames.first.isEmpty()) {
+                // Otherwise... recurse.
+                Pair<List<String>, Integer> parentFieldNames =
+                        !isIndexOnFunction
+                                ? getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignAndExpressionIndexes[0],
+                                        assignAndExpressionIndexes[1], funcVarIndex, parentFuncExpr, context)
+                                : NO_FIELD_NAME;
+
+                if (parentFieldNames.first.isEmpty() && !isIndexOnFunction) {
                     //Nested assign was not a field access.
                     //We will not use index
                     return NO_FIELD_NAME;
@@ -3056,13 +3107,22 @@
                     optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
                 }
 
-                //add fieldName to the nested fieldName, return
-                if (nestedAccessFieldName != null) {
-                    parentFieldNames.first.addAll(nestedAccessFieldName);
+                if (!isIndexOnFunction) {
+                    //add fieldName to the nested fieldName, return
+                    if (nestedAccessFieldName != null) {
+                        parentFieldNames.first.addAll(nestedAccessFieldName);
+                    } else {
+                        parentFieldNames.first.add(fieldName);
+                    }
+                    return (parentFieldNames);
+
                 } else {
-                    parentFieldNames.first.add(fieldName);
+                    if (nestedAccessFieldName != null) {
+                        return new Pair<>(nestedAccessFieldName, sourceIndicator.getValue());
+                    } else {
+                        return new Pair<>(new ArrayList<>(List.of(fieldName)), sourceIndicator.getValue());
+                    }
                 }
-                return (parentFieldNames);
             }
 
             if (optFuncExpr != null) {
@@ -3199,4 +3259,73 @@
         return funId.equals(FIELD_ACCESS_BY_NAME) || funId.equals(FIELD_ACCESS_BY_INDEX)
                 || funId.equals(FIELD_ACCESS_NESTED);
     }
+
+    /**
+     * If we are accessing some field through a function application (or series of function applications) of the
+     * following:
+     * <p><pre>
+     * | OBJECT_ADD    | OBJECT_REMOVE | OBJECT_ADD_FIELDS    |
+     * | OBJECT_CONCAT | OBJECT_RENAME | OBJECT_REMOVE_FIELDS |
+     * </pre>
+     * ...then we still might be able to use an index. Check the output type of applying our function(s) and verify
+     * that the input is a data source variable.
+     */
+    public static boolean canUseIndexOnFunction(AbstractFunctionCallExpression funcExpr, MutableInt sourceIndicator,
+            Collection<LogicalVariable> foundDatasourceVariables, IOptimizableFuncExpr optFuncExpr,
+            IVariableTypeEnvironment typeEnv, IOptimizationContext context) throws AlgebricksException {
+        FunctionIdentifier functionID = funcExpr.getFunctionIdentifier();
+        if (!INDEX_USE_ON_FUNCTION_CALL_WHITELIST.containsKey(functionID)) {
+            return false;
+        }
+
+        // Our output should be an object (this is more of a sanity check given that we have a whitelist).
+        IExpressionTypeComputer expressionTypeComputer = context.getExpressionTypeComputer();
+        IMetadataProvider<?, ?> metadataProvider = context.getMetadataProvider();
+        IAType originalOutputType = (IAType) expressionTypeComputer.getType(funcExpr, metadataProvider, typeEnv);
+        IAType outputType = TypeComputeUtils.getActualType(originalOutputType);
+        ARecordType outputRecordType = TypeComputeUtils.extractRecordType(outputType);
+        if (outputRecordType == null) {
+            return false;
+        }
+
+        // Check the type of our input, according to record variables in each function's argument.
+        boolean isDataSourceVariableFound = false;
+        Set<Integer> indicesToCheck = INDEX_USE_ON_FUNCTION_CALL_WHITELIST.get(functionID);
+        if (indicesToCheck.equals(ALL_INDEX_FUNCTION_ARGUMENTS)) {
+            indicesToCheck = IntStream.range(0, funcExpr.getArguments().size()).boxed().collect(Collectors.toSet());
+        }
+        for (Integer functionCallArgumentIndex : indicesToCheck) {
+            ILogicalExpression inputRecordExpr = funcExpr.getArguments().get(functionCallArgumentIndex).getValue();
+            switch (inputRecordExpr.getExpressionTag()) {
+                case FUNCTION_CALL:
+                    AbstractFunctionCallExpression arg0FuncExpr = (AbstractFunctionCallExpression) inputRecordExpr;
+                    isDataSourceVariableFound |= canUseIndexOnFunction(arg0FuncExpr, sourceIndicator,
+                            foundDatasourceVariables, optFuncExpr, typeEnv, context);
+                    break;
+
+                case VARIABLE:
+                    // Base case. We should be using a data source variable here.
+                    VariableReferenceExpression inputRecordVarExpr = (VariableReferenceExpression) inputRecordExpr;
+                    LogicalVariable inputRecordVar = inputRecordVarExpr.getVariableReference();
+                    if (optFuncExpr != null) {
+                        for (int i = 0; i < optFuncExpr.getNumLogicalVars(); i++) {
+                            OptimizableOperatorSubTree operatorSubTree = optFuncExpr.getOperatorSubTree(i);
+                            if (operatorSubTree == null) {
+                                continue;
+                            }
+                            if (operatorSubTree.getDataSourceVariables().stream().anyMatch(inputRecordVar::equals)) {
+                                OptimizableOperatorSubTree.RecordTypeSource recordTypeSource =
+                                        operatorSubTree.getRecordTypeFor(inputRecordVar);
+                                foundDatasourceVariables.add(inputRecordVar);
+                                sourceIndicator.setValue(recordTypeSource.sourceIndicator);
+                                isDataSourceVariableFound = true;
+                                break;
+                            }
+                        }
+                    }
+                    break;
+            }
+        }
+        return isDataSourceVariableFound;
+    }
 }
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 acd557e..1aa2bee 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
@@ -537,9 +537,7 @@
                 if (splitIntoConjuncts(conjunct.getValue(), innerExprConjuncts)) {
                     conjuncts.addAll(innerExprConjuncts);
                 } else {
-
                     conjuncts.add(new MutableObject<>(conjunct.getValue()));
-
                 }
             }
             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
index 0889856..81b18df 100644
--- 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
@@ -154,7 +154,7 @@
         PlanNode cheapestPlanNode = joinEnum.allPlans.get(cheapestPlan);
 
         if (numberOfFromTerms > 1) {
-            buildNewTree(cheapestPlanNode, joinLeafInputsHashMap, joinOps, new MutableInt(0));
+            buildNewTree(cheapestPlanNode, joinLeafInputsHashMap, joinOps, new MutableInt(0), context);
             printPlan(pp, (AbstractLogicalOperator) joinOps.get(0), "New Whole Plan after buildNewTree 1");
             ILogicalOperator root = addConstantInternalEdgesAtTheTop(joinOps.get(0), internalEdges);
             printPlan(pp, (AbstractLogicalOperator) joinOps.get(0), "New Whole Plan after buildNewTree 2");
@@ -443,7 +443,8 @@
 
     // This one is for join queries
     private void buildNewTree(PlanNode plan, HashMap<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap,
-            List<ILogicalOperator> joinOps, MutableInt totalNumberOfJoins) {
+            List<ILogicalOperator> joinOps, MutableInt totalNumberOfJoins, IOptimizationContext context)
+            throws AlgebricksException {
         // 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();
@@ -499,13 +500,15 @@
                 addCardCostAnnotations(selOp, leftPlan);
             }
             joinOp.getInputs().get(0).setValue(leftInput);
+            context.computeAndSetTypeEnvironmentForOperator(joinOp.getInputs().get(0).getValue());
             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);
+            context.computeAndSetTypeEnvironmentForOperator(joinOp.getInputs().get(0).getValue());
+            buildNewTree(allPlans.get(leftIndex), joinLeafInputsHashMap, joinOps, totalNumberOfJoins, context);
         }
 
         if (rightPlan.IsScanNode()) {
@@ -517,13 +520,15 @@
                 addCardCostAnnotations(selOp, rightPlan);
             }
             joinOp.getInputs().get(1).setValue(rightInput);
+            context.computeAndSetTypeEnvironmentForOperator(joinOp.getInputs().get(1).getValue());
             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);
+            context.computeAndSetTypeEnvironmentForOperator(joinOp.getInputs().get(1).getValue());
+            buildNewTree(allPlans.get(rightIndex), joinLeafInputsHashMap, joinOps, totalNumberOfJoins, context);
         }
     }
 
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
index 3943cf4..822d824 100644
--- 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
@@ -54,6 +54,7 @@
 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.SwitchOperator;
 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;
@@ -169,6 +170,11 @@
     }
 
     @Override
+    public Pair<Double, Double> visitSwitchOperator(SwitchOperator 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);
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
index e7a620d..f23af33 100644
--- 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
@@ -29,8 +29,11 @@
 
 import org.apache.asterix.common.annotations.IndexedNLJoinExpressionAnnotation;
 import org.apache.asterix.common.annotations.SecondaryIndexSearchPreferenceAnnotation;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.DataSourceId;
+import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.om.base.AOrderedList;
 import org.apache.asterix.om.base.IAObject;
@@ -69,6 +72,7 @@
 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.hyracks.control.common.config.OptionTypes;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -76,6 +80,14 @@
 
     private static final Logger LOGGER = LogManager.getLogger();
 
+    // Number of levels to do full join and plan enumeration
+    public static final String CBO_FULL_ENUM_LEVEL_KEY = "cbofullenumlevel";
+    private static final int CBO_FULL_ENUM_LEVEL_DEFAULT = 0;
+
+    // Mode for cartesian product plan generation during join and plan enumeration
+    public static final String CBO_CP_ENUM_KEY = "cbocpenum";
+    private static final boolean CBO_CP_ENUM_DEFAULT = true;
+
     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
@@ -92,6 +104,9 @@
     protected PhysicalOptimizationConfig physOptConfig;
     protected boolean cboMode;
     protected boolean cboTestMode;
+
+    protected int cboFullEnumLevel;
+    protected boolean cboCPEnumMode;
     protected int numberOfTerms;
     protected AbstractLogicalOperator op;
     protected boolean connectedJoinGraph;
@@ -107,7 +122,8 @@
             List<Pair<EmptyTupleSourceOperator, DataSourceScanOperator>> emptyTupleAndDataSourceOps,
             Map<EmptyTupleSourceOperator, ILogicalOperator> joinLeafInputsHashMap,
             Map<DataSourceScanOperator, EmptyTupleSourceOperator> dataSourceEmptyTupleHashMap,
-            List<ILogicalOperator> internalEdges, List<ILogicalOperator> joinOps, IOptimizationContext context) {
+            List<ILogicalOperator> internalEdges, List<ILogicalOperator> joinOps, IOptimizationContext context)
+            throws AsterixException {
         this.singleDatasetPreds = new ArrayList<>();
         this.joinConditions = new ArrayList<>();
         this.internalEdges = new ArrayList<>();
@@ -115,6 +131,8 @@
         this.numberOfTerms = numberOfFromTerms;
         this.cboMode = cboMode;
         this.cboTestMode = cboTestMode;
+        this.cboFullEnumLevel = getCBOFullEnumLevel(context);
+        this.cboCPEnumMode = getCBOCPEnumMode(context);
         this.connectedJoinGraph = true;
         this.optCtx = context;
         this.physOptConfig = context.getPhysicalOptimizationConfig();
@@ -141,6 +159,24 @@
         }
     }
 
+    private int getCBOFullEnumLevel(IOptimizationContext context) throws AsterixException {
+        MetadataProvider mdp = (MetadataProvider) context.getMetadataProvider();
+
+        String valueInQuery = mdp.getProperty(CBO_FULL_ENUM_LEVEL_KEY);
+        try {
+            return valueInQuery == null ? CBO_FULL_ENUM_LEVEL_DEFAULT
+                    : OptionTypes.POSITIVE_INTEGER.parse(valueInQuery);
+        } catch (IllegalArgumentException e) {
+            throw AsterixException.create(ErrorCode.COMPILATION_BAD_QUERY_PARAMETER_VALUE, CBO_FULL_ENUM_LEVEL_KEY, 1,
+                    "");
+        }
+    }
+
+    private boolean getCBOCPEnumMode(IOptimizationContext context) {
+        MetadataProvider mdp = (MetadataProvider) context.getMetadataProvider();
+        return mdp.getBooleanProperty(CBO_CP_ENUM_KEY, CBO_CP_ENUM_DEFAULT);
+    }
+
     public List<JoinCondition> getJoinConditions() {
         return joinConditions;
     }
@@ -618,11 +654,11 @@
                 JoinNode jnIJ = jnArray[addPlansToThisJn];
                 jnIJ.jnArrayIndex = addPlansToThisJn;
                 jnIJ.addMultiDatasetPlans(jnI, jnJ);
-                if (forceJoinOrderMode) {
+                if (forceJoinOrderMode && level > cboFullEnumLevel) {
                     break;
                 }
             }
-            if (forceJoinOrderMode) {
+            if (forceJoinOrderMode && level > cboFullEnumLevel) {
                 break;
             }
         }
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
index 1f9300d..4418e4b 100644
--- 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
@@ -283,12 +283,13 @@
     public int addSingleDatasetPlans() {
         List<PlanNode> allPlans = joinEnum.allPlans;
         ICost opCost, totalCost;
-
+        PlanNode pn, cheapestPlan;
         opCost = joinEnum.getCostMethodsHandle().costFullScan(this);
         totalCost = opCost;
-        if (this.cheapestPlanIndex == PlanNode.NO_PLAN || opCost.costLT(this.cheapestPlanCost)) {
+        boolean forceEnum = level <= joinEnum.cboFullEnumLevel;
+        if (this.cheapestPlanIndex == PlanNode.NO_PLAN || opCost.costLT(this.cheapestPlanCost) || forceEnum) {
             // for now just add one plan
-            PlanNode pn = new PlanNode(allPlans.size(), joinEnum);
+            pn = new PlanNode(allPlans.size(), joinEnum);
             pn.jn = this;
             pn.datasetName = this.datasetNames.get(0);
             pn.correspondingEmptyTupleSourceOp = this.correspondingEmptyTupleSourceOp;
@@ -301,10 +302,15 @@
             pn.totalCost = totalCost;
 
             allPlans.add(pn);
-            this.planIndexesArray.add(allPlans.size() - 1);
-            this.cheapestPlanCost = totalCost;
-            this.cheapestPlanIndex = allPlans.size() - 1;
-            return this.cheapestPlanIndex;
+            this.planIndexesArray.add(pn.allPlansIndex);
+            if (!forceEnum) {
+                cheapestPlan = pn;
+            } else {
+                cheapestPlan = findCheapestPlan();
+            }
+            this.cheapestPlanCost = cheapestPlan.totalCost;
+            this.cheapestPlanIndex = cheapestPlan.allPlansIndex;
+            return pn.allPlansIndex;
         }
         return PlanNode.NO_PLAN;
     }
@@ -312,12 +318,14 @@
     protected void buildIndexPlan(boolean forceIndexPlan) {
         List<PlanNode> allPlans = joinEnum.allPlans;
         ICost opCost, totalCost;
-
+        PlanNode pn, cheapestPlan;
         opCost = joinEnum.getCostMethodsHandle().costIndexScan(this);
         totalCost = opCost;
-        if (this.cheapestPlanIndex == PlanNode.NO_PLAN || opCost.costLT(this.cheapestPlanCost) || forceIndexPlan) {
+        boolean forceEnum = level <= joinEnum.cboFullEnumLevel;
+        if (this.cheapestPlanIndex == PlanNode.NO_PLAN || opCost.costLT(this.cheapestPlanCost) || forceIndexPlan
+                || forceEnum) {
             // for now just add one plan
-            PlanNode pn = new PlanNode(allPlans.size(), joinEnum);
+            pn = new PlanNode(allPlans.size(), joinEnum);
             pn.jn = this;
             pn.datasetName = this.datasetNames.get(0);
             pn.correspondingEmptyTupleSourceOp = this.correspondingEmptyTupleSourceOp;
@@ -327,12 +335,18 @@
             pn.planIndexes[1] = PlanNode.NO_PLAN; // There ane no plans below this plan.
             pn.opCost = opCost;
             pn.scanOp = PlanNode.ScanMethod.INDEX_SCAN;
+            pn.indexHint = forceIndexPlan;
             pn.totalCost = totalCost;
 
             allPlans.add(pn);
-            this.planIndexesArray.add(allPlans.size() - 1);
-            this.cheapestPlanCost = totalCost;
-            this.cheapestPlanIndex = allPlans.size() - 1;
+            this.planIndexesArray.add(pn.allPlansIndex);
+            if (!forceEnum) {
+                cheapestPlan = pn;
+            } else {
+                cheapestPlan = findCheapestPlan();
+            }
+            this.cheapestPlanCost = cheapestPlan.totalCost;
+            this.cheapestPlanIndex = cheapestPlan.allPlansIndex;
         }
     }
 
@@ -463,50 +477,49 @@
         }
     }
 
-    protected int buildHashJoinPlan(JoinNode leftJn, JoinNode rightJn, ILogicalExpression hashJoinExpr,
-            HashJoinExpressionAnnotation hintHashJoin) {
+    protected int buildHashJoinPlan(JoinNode leftJn, JoinNode rightJn, PlanNode leftPlan, PlanNode rightPlan,
+            ILogicalExpression hashJoinExpr, HashJoinExpressionAnnotation hintHashJoin) {
         List<PlanNode> allPlans = joinEnum.allPlans;
-        PlanNode pn;
+        PlanNode pn, cheapestPlan;
         ICost hjCost, leftExchangeCost, rightExchangeCost, 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()) {
+        if (joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_LEFTDEEP) && !leftJn.IsBaseLevelJoinNode()
+                && level > joinEnum.cboFullEnumLevel) {
             return PlanNode.NO_PLAN;
         }
 
         if (joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_RIGHTDEEP)
-                && !rightJn.IsBaseLevelJoinNode()) {
+                && !rightJn.IsBaseLevelJoinNode() && level > joinEnum.cboFullEnumLevel) {
             return PlanNode.NO_PLAN;
         }
 
-        if (rightJn.cardinality * rightJn.size <= leftJn.cardinality * leftJn.size || hintHashJoin != null
-                || joinEnum.forceJoinOrderMode
-                || !joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_ZIGZAG)) {
+        boolean forceEnum = hintHashJoin != null || joinEnum.forceJoinOrderMode
+                || !joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_ZIGZAG)
+                || level <= joinEnum.cboFullEnumLevel;
+
+        if (rightJn.cardinality * rightJn.size <= leftJn.cardinality * leftJn.size || forceEnum) {
             // We want to build with the smaller side.
             hjCost = joinEnum.getCostMethodsHandle().costHashJoin(this);
             leftExchangeCost = joinEnum.getCostMethodsHandle().computeHJProbeExchangeCost(this);
             rightExchangeCost = joinEnum.getCostMethodsHandle().computeHJBuildExchangeCost(this);
-            childCosts = allPlans.get(leftPlan).totalCost.costAdd(allPlans.get(rightPlan).totalCost);
+            childCosts = allPlans.get(leftPlan.allPlansIndex).totalCost
+                    .costAdd(allPlans.get(rightPlan.allPlansIndex).totalCost);
             totalCost = hjCost.costAdd(leftExchangeCost).costAdd(rightExchangeCost).costAdd(childCosts);
-            if (this.cheapestPlanIndex == PlanNode.NO_PLAN || totalCost.costLT(this.cheapestPlanCost)) {
+            if (this.cheapestPlanIndex == PlanNode.NO_PLAN || totalCost.costLT(this.cheapestPlanCost) || forceEnum) {
                 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.planIndexes[0] = leftPlan.allPlansIndex;
+                pn.planIndexes[1] = rightPlan.allPlansIndex;
                 pn.joinOp = PlanNode.JoinMethod.HYBRID_HASH_JOIN; // need to check that all the conditions have equality predicates ONLY.
-                if (hintHashJoin != null) {
-                    hintHashJoin.setBuildSide(HashJoinExpressionAnnotation.BuildSide.RIGHT);
-                }
+                pn.joinHint = hintHashJoin;
                 pn.side = HashJoinExpressionAnnotation.BuildSide.RIGHT;
                 pn.joinExpr = hashJoinExpr;
                 pn.opCost = hjCost;
@@ -514,61 +527,64 @@
                 pn.leftExchangeCost = leftExchangeCost;
                 pn.rightExchangeCost = rightExchangeCost;
                 allPlans.add(pn);
-                this.planIndexesArray.add(allPlans.size() - 1);
-                this.cheapestPlanCost = totalCost;
-                this.cheapestPlanIndex = allPlans.size() - 1;
-                return this.cheapestPlanIndex;
+                this.planIndexesArray.add(pn.allPlansIndex);
+                if (!forceEnum) {
+                    cheapestPlan = pn;
+                } else {
+                    cheapestPlan = findCheapestPlan();
+                }
+                this.cheapestPlanCost = cheapestPlan.totalCost;
+                this.cheapestPlanIndex = cheapestPlan.allPlansIndex;
+                return pn.allPlansIndex;
             }
         }
 
         return PlanNode.NO_PLAN;
     }
 
-    protected int buildBroadcastHashJoinPlan(JoinNode leftJn, JoinNode rightJn, ILogicalExpression hashJoinExpr,
-            BroadcastExpressionAnnotation hintBroadcastHashJoin) {
+    protected int buildBroadcastHashJoinPlan(JoinNode leftJn, JoinNode rightJn, PlanNode leftPlan, PlanNode rightPlan,
+            ILogicalExpression hashJoinExpr, BroadcastExpressionAnnotation hintBroadcastHashJoin) {
         List<PlanNode> allPlans = joinEnum.allPlans;
-        PlanNode pn;
+        PlanNode pn, cheapestPlan;
         ICost bcastHjCost, leftExchangeCost, rightExchangeCost, 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()) {
+        if (joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_LEFTDEEP) && !leftJn.IsBaseLevelJoinNode()
+                && level > joinEnum.cboFullEnumLevel) {
             return PlanNode.NO_PLAN;
         }
 
         if (joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_RIGHTDEEP)
-                && !rightJn.IsBaseLevelJoinNode()) {
+                && !rightJn.IsBaseLevelJoinNode() && level > joinEnum.cboFullEnumLevel) {
             return PlanNode.NO_PLAN;
         }
 
-        if (rightJn.cardinality * rightJn.size <= leftJn.cardinality * leftJn.size || hintBroadcastHashJoin != null
-                || joinEnum.forceJoinOrderMode
-                || !joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_ZIGZAG)) {
+        boolean forceEnum = hintBroadcastHashJoin != null || joinEnum.forceJoinOrderMode
+                || !joinEnum.queryPlanShape.equals(AlgebricksConfig.QUERY_PLAN_SHAPE_ZIGZAG)
+                || level <= joinEnum.cboFullEnumLevel;
+        if (rightJn.cardinality * rightJn.size <= leftJn.cardinality * leftJn.size || forceEnum) {
             // We want to broadcast and build with the smaller side.
             bcastHjCost = joinEnum.getCostMethodsHandle().costBroadcastHashJoin(this);
             leftExchangeCost = joinEnum.getCostHandle().zeroCost();
             rightExchangeCost = joinEnum.getCostMethodsHandle().computeBHJBuildExchangeCost(this);
-            childCosts = allPlans.get(leftPlan).totalCost.costAdd(allPlans.get(rightPlan).totalCost);
+            childCosts = allPlans.get(leftPlan.allPlansIndex).totalCost
+                    .costAdd(allPlans.get(rightPlan.allPlansIndex).totalCost);
             totalCost = bcastHjCost.costAdd(rightExchangeCost).costAdd(childCosts);
-            if (this.cheapestPlanIndex == PlanNode.NO_PLAN || totalCost.costLT(this.cheapestPlanCost)) {
+            if (this.cheapestPlanIndex == PlanNode.NO_PLAN || totalCost.costLT(this.cheapestPlanCost) || forceEnum) {
                 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.planIndexes[0] = leftPlan.allPlansIndex;
+                pn.planIndexes[1] = rightPlan.allPlansIndex;
                 pn.joinOp = PlanNode.JoinMethod.BROADCAST_HASH_JOIN; // need to check that all the conditions have equality predicates ONLY.
-                if (hintBroadcastHashJoin != null) {
-                    hintBroadcastHashJoin.setBroadcastSide(BroadcastExpressionAnnotation.BroadcastSide.RIGHT);
-                }
+                pn.joinHint = hintBroadcastHashJoin;
                 pn.side = HashJoinExpressionAnnotation.BuildSide.RIGHT;
                 pn.joinExpr = hashJoinExpr;
                 pn.opCost = bcastHjCost;
@@ -577,29 +593,34 @@
                 pn.rightExchangeCost = rightExchangeCost;
 
                 allPlans.add(pn);
-                this.planIndexesArray.add(allPlans.size() - 1);
-                this.cheapestPlanCost = totalCost;
-                this.cheapestPlanIndex = allPlans.size() - 1;
-                return this.cheapestPlanIndex;
+                this.planIndexesArray.add(pn.allPlansIndex);
+                if (!forceEnum) {
+                    cheapestPlan = pn;
+                } else {
+                    cheapestPlan = findCheapestPlan();
+                }
+                this.cheapestPlanCost = cheapestPlan.totalCost;
+                this.cheapestPlanIndex = cheapestPlan.allPlansIndex;
+                return pn.allPlansIndex;
             }
         }
 
         return PlanNode.NO_PLAN;
     }
 
-    protected int buildNLJoinPlan(JoinNode leftJn, JoinNode rightJn, ILogicalExpression nestedLoopJoinExpr)
+    protected int buildNLJoinPlan(JoinNode leftJn, JoinNode rightJn, PlanNode leftPlan, PlanNode rightPlan,
+            ILogicalExpression nestedLoopJoinExpr, IndexedNLJoinExpressionAnnotation hintNLJoin)
             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;
+        PlanNode pn, cheapestPlan;
         ICost nljCost, leftExchangeCost, rightExchangeCost, 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.
@@ -612,42 +633,50 @@
         nljCost = joinEnum.getCostMethodsHandle().costIndexNLJoin(this);
         leftExchangeCost = joinEnum.getCostMethodsHandle().computeNLJOuterExchangeCost(this);
         rightExchangeCost = joinEnum.getCostHandle().zeroCost();
-        childCosts = allPlans.get(leftPlan).totalCost;
+        childCosts = allPlans.get(leftPlan.allPlansIndex).totalCost;
         totalCost = nljCost.costAdd(leftExchangeCost).costAdd(childCosts);
-        if (this.cheapestPlanIndex == PlanNode.NO_PLAN || totalCost.costLT(this.cheapestPlanCost)) {
+        boolean forceEnum = hintNLJoin != null || joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel;
+        if (this.cheapestPlanIndex == PlanNode.NO_PLAN || totalCost.costLT(this.cheapestPlanCost) || forceEnum) {
             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.planIndexes[0] = leftPlan.allPlansIndex;
+            pn.planIndexes[1] = rightPlan.allPlansIndex;
             pn.joinOp = PlanNode.JoinMethod.INDEX_NESTED_LOOP_JOIN;
+            pn.joinHint = hintNLJoin;
             pn.joinExpr = nestedLoopJoinExpr; // save it so can be used to add the NESTED annotation in getNewTree.
             pn.opCost = nljCost;
             pn.totalCost = totalCost;
             pn.leftExchangeCost = leftExchangeCost;
             pn.rightExchangeCost = rightExchangeCost;
-
             allPlans.add(pn);
-            this.planIndexesArray.add(allPlans.size() - 1);
-            this.cheapestPlanCost = totalCost;
-            this.cheapestPlanIndex = allPlans.size() - 1;
-            return allPlans.size() - 1;
+            this.planIndexesArray.add(pn.allPlansIndex);
+            if (!forceEnum) {
+                cheapestPlan = pn;
+            } else {
+                cheapestPlan = findCheapestPlan();
+            }
+            this.cheapestPlanCost = cheapestPlan.totalCost;
+            this.cheapestPlanIndex = cheapestPlan.allPlansIndex;
+            return pn.allPlansIndex;
         }
         return PlanNode.NO_PLAN;
     }
 
-    protected int buildCPJoinPlan(JoinNode leftJn, JoinNode rightJn, ILogicalExpression hashJoinExpr,
-            ILogicalExpression nestedLoopJoinExpr) {
+    protected int buildCPJoinPlan(JoinNode leftJn, JoinNode rightJn, PlanNode leftPlan, PlanNode rightPlan,
+            ILogicalExpression hashJoinExpr, ILogicalExpression nestedLoopJoinExpr) {
         // Now build a cartesian product nested loops plan
         List<PlanNode> allPlans = joinEnum.allPlans;
-        PlanNode pn;
+        PlanNode pn, cheapestPlan;
         ICost cpCost, leftExchangeCost, rightExchangeCost, childCosts, totalCost;
 
+        if (!joinEnum.cboCPEnumMode) {
+            return PlanNode.NO_PLAN;
+        }
+
         this.leftJn = leftJn;
         this.rightJn = rightJn;
-        int leftPlan = leftJn.cheapestPlanIndex;
-        int rightPlan = rightJn.cheapestPlanIndex;
 
         ILogicalExpression cpJoinExpr = null;
         List<Integer> newJoinConditions = this.getNewJoinConditionsOnly();
@@ -670,58 +699,93 @@
         cpCost = joinEnum.getCostMethodsHandle().costCartesianProductJoin(this);
         leftExchangeCost = joinEnum.getCostHandle().zeroCost();
         rightExchangeCost = joinEnum.getCostMethodsHandle().computeCPRightExchangeCost(this);
-        childCosts = allPlans.get(leftPlan).totalCost.costAdd(allPlans.get(rightPlan).totalCost);
+        childCosts =
+                allPlans.get(leftPlan.allPlansIndex).totalCost.costAdd(allPlans.get(rightPlan.allPlansIndex).totalCost);
         totalCost = cpCost.costAdd(rightExchangeCost).costAdd(childCosts);
-        if (this.cheapestPlanIndex == PlanNode.NO_PLAN || totalCost.costLT(this.cheapestPlanCost)) {
+        boolean forceEnum = joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel;
+        if (this.cheapestPlanIndex == PlanNode.NO_PLAN || totalCost.costLT(this.cheapestPlanCost) || forceEnum) {
             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.planIndexes[0] = leftPlan.allPlansIndex;
+            pn.planIndexes[1] = rightPlan.allPlansIndex;
             pn.joinOp = PlanNode.JoinMethod.CARTESIAN_PRODUCT_JOIN;
             pn.joinExpr = Objects.requireNonNullElse(cpJoinExpr, ConstantExpression.TRUE);
             pn.opCost = cpCost;
             pn.totalCost = totalCost;
             pn.leftExchangeCost = leftExchangeCost;
             pn.rightExchangeCost = rightExchangeCost;
-
             allPlans.add(pn);
-            this.planIndexesArray.add(allPlans.size() - 1);
-            this.cheapestPlanCost = totalCost;
-            this.cheapestPlanIndex = allPlans.size() - 1;
-            return allPlans.size() - 1;
+            this.planIndexesArray.add(pn.allPlansIndex);
+            if (!forceEnum) {
+                cheapestPlan = pn;
+            } else {
+                cheapestPlan = findCheapestPlan();
+            }
+            this.cheapestPlanCost = cheapestPlan.totalCost;
+            this.cheapestPlanIndex = cheapestPlan.allPlansIndex;
+            return pn.allPlansIndex;
         }
         return PlanNode.NO_PLAN;
     }
 
-    protected Pair<Integer, ICost> addMultiDatasetPlans(JoinNode leftJn, JoinNode rightJn) throws AlgebricksException {
+    protected void addMultiDatasetPlans(JoinNode leftJn, JoinNode rightJn) throws AlgebricksException {
+        PlanNode leftPlan, rightPlan;
+
+        if (level > joinEnum.cboFullEnumLevel) {
+            // FOR JOIN NODE LEVELS GREATER THAN THE LEVEL SPECIFIED FOR FULL ENUMERATION,
+            // DO NOT DO FULL ENUMERATION => PRUNE
+            if (leftJn.cheapestPlanIndex == PlanNode.NO_PLAN || rightJn.cheapestPlanIndex == PlanNode.NO_PLAN) {
+                return;
+            }
+            leftPlan = joinEnum.allPlans.get(leftJn.cheapestPlanIndex);
+            rightPlan = joinEnum.allPlans.get(rightJn.cheapestPlanIndex);
+            addMultiDatasetPlans(leftJn, rightJn, leftPlan, rightPlan);
+        } else {
+            // FOR JOIN NODE LEVELS LESS THAN OR EQUAL TO THE LEVEL SPECIFIED FOR FULL ENUMERATION,
+            // DO FULL ENUMERATION => DO NOT PRUNE
+            for (int leftPlanIndex : leftJn.planIndexesArray) {
+                leftPlan = joinEnum.allPlans.get(leftPlanIndex);
+                for (int rightPlanIndex : rightJn.planIndexesArray) {
+                    rightPlan = joinEnum.allPlans.get(rightPlanIndex);
+                    addMultiDatasetPlans(leftJn, rightJn, leftPlan, rightPlan);
+                }
+            }
+        }
+    }
+
+    protected void addMultiDatasetPlans(JoinNode leftJn, JoinNode rightJn, PlanNode leftPlan, PlanNode rightPlan)
+            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);
+            return;
         }
 
         if (this.cardinality >= Cost.MAX_CARD) {
-            return new Pair<>(PlanNode.NO_PLAN, noJoinCost); // no card hint available, so do not add this plan
+            return; // no card available, so do not add this plan
+        }
+
+        if (leftJn.cheapestPlanIndex == PlanNode.NO_PLAN || rightJn.cheapestPlanIndex == PlanNode.NO_PLAN) {
+            return;
         }
 
         List<Integer> newJoinConditions = this.getNewJoinConditionsOnly(); // these will be a subset of applicable join conditions.
+        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 && level > joinEnum.cboFullEnumLevel) {
+                return;
+            }
+        }
         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
+            return; // no card available, so do not add this plan
         }
 
         int hjPlan, commutativeHjPlan, bcastHjPlan, commutativeBcastHjPlan, nljPlan, commutativeNljPlan, cpPlan,
@@ -733,10 +797,6 @@
         BroadcastExpressionAnnotation hintBroadcastHashJoin = joinEnum.findBroadcastHashJoinHint(newJoinConditions);
         IndexedNLJoinExpressionAnnotation 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) {
             boolean build = (hintHashJoin.getBuildOrProbe() == HashJoinExpressionAnnotation.BuildOrProbe.BUILD);
             boolean probe = (hintHashJoin.getBuildOrProbe() == HashJoinExpressionAnnotation.BuildOrProbe.PROBE);
@@ -752,12 +812,13 @@
                         || rightJn.aliases.contains(buildOrProbeObject)))
                         || (probe && (leftJn.datasetNames.contains(buildOrProbeObject)
                                 || leftJn.aliases.contains(buildOrProbeObject)))) {
-                    hjPlan = buildHashJoinPlan(leftJn, rightJn, hashJoinExpr, hintHashJoin);
+                    hjPlan = buildHashJoinPlan(leftJn, rightJn, leftPlan, rightPlan, hashJoinExpr, hintHashJoin);
                 } else if ((build && (leftJn.datasetNames.contains(buildOrProbeObject)
                         || leftJn.aliases.contains(buildOrProbeObject)))
                         || (probe && (rightJn.datasetNames.contains(buildOrProbeObject)
                                 || rightJn.aliases.contains(buildOrProbeObject)))) {
-                    commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, hashJoinExpr, hintHashJoin);
+                    commutativeHjPlan =
+                            buildHashJoinPlan(rightJn, leftJn, rightPlan, leftPlan, hashJoinExpr, hintHashJoin);
                 }
             } else {
                 // Hints are attached to predicates, so newJoinConditions should not be empty, but adding the check to be safe.
@@ -771,21 +832,24 @@
                                 (build ? "build " : "probe ") + "with " + buildOrProbeObject));
                     }
                 }
-                hjPlan = buildHashJoinPlan(leftJn, rightJn, hashJoinExpr, null);
-                if (!joinEnum.forceJoinOrderMode) {
-                    commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, hashJoinExpr, null);
+                hjPlan = buildHashJoinPlan(leftJn, rightJn, leftPlan, rightPlan, hashJoinExpr, null);
+                if (!joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel) {
+                    commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, rightPlan, leftPlan, hashJoinExpr, null);
                 }
-                bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, hashJoinExpr, null);
-                if (!joinEnum.forceJoinOrderMode) {
-                    commutativeBcastHjPlan = buildBroadcastHashJoinPlan(rightJn, leftJn, hashJoinExpr, null);
+                bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, leftPlan, rightPlan, hashJoinExpr, null);
+                if (!joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel) {
+                    commutativeBcastHjPlan =
+                            buildBroadcastHashJoinPlan(rightJn, leftJn, rightPlan, leftPlan, hashJoinExpr, null);
                 }
-                nljPlan = buildNLJoinPlan(leftJn, rightJn, nestedLoopJoinExpr);
-                if (!joinEnum.forceJoinOrderMode) {
-                    commutativeNljPlan = buildNLJoinPlan(rightJn, leftJn, nestedLoopJoinExpr);
+                nljPlan = buildNLJoinPlan(leftJn, rightJn, leftPlan, rightPlan, nestedLoopJoinExpr, null);
+                if (!joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel) {
+                    commutativeNljPlan =
+                            buildNLJoinPlan(rightJn, leftJn, rightPlan, leftPlan, nestedLoopJoinExpr, null);
                 }
-                cpPlan = buildCPJoinPlan(leftJn, rightJn, hashJoinExpr, nestedLoopJoinExpr);
-                if (!joinEnum.forceJoinOrderMode) {
-                    commutativeCpPlan = buildCPJoinPlan(rightJn, leftJn, hashJoinExpr, nestedLoopJoinExpr);
+                cpPlan = buildCPJoinPlan(leftJn, rightJn, leftPlan, rightPlan, hashJoinExpr, nestedLoopJoinExpr);
+                if (!joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel) {
+                    commutativeCpPlan =
+                            buildCPJoinPlan(rightJn, leftJn, rightPlan, leftPlan, hashJoinExpr, nestedLoopJoinExpr);
                 }
             }
         } else if (hintBroadcastHashJoin != null) {
@@ -798,16 +862,18 @@
             }
             if (validBroadcastObject) {
                 if (rightJn.datasetNames.contains(broadcastObject) || rightJn.aliases.contains(broadcastObject)) {
-                    bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, hashJoinExpr, hintBroadcastHashJoin);
+                    bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, leftPlan, rightPlan, hashJoinExpr,
+                            hintBroadcastHashJoin);
                 } else if (leftJn.datasetNames.contains(broadcastObject) || leftJn.aliases.contains(broadcastObject)) {
-                    commutativeBcastHjPlan =
-                            buildBroadcastHashJoinPlan(rightJn, leftJn, hashJoinExpr, hintBroadcastHashJoin);
+                    commutativeBcastHjPlan = buildBroadcastHashJoinPlan(rightJn, leftJn, rightPlan, leftPlan,
+                            hashJoinExpr, hintBroadcastHashJoin);
                 }
             } else if (broadcastObject == null) {
-                bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, hashJoinExpr, hintBroadcastHashJoin);
-                if (!joinEnum.forceJoinOrderMode) {
-                    commutativeBcastHjPlan =
-                            buildBroadcastHashJoinPlan(rightJn, leftJn, hashJoinExpr, hintBroadcastHashJoin);
+                bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, leftPlan, rightPlan, hashJoinExpr,
+                        hintBroadcastHashJoin);
+                if (!joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel) {
+                    commutativeBcastHjPlan = buildBroadcastHashJoinPlan(rightJn, leftJn, rightPlan, leftPlan,
+                            hashJoinExpr, hintBroadcastHashJoin);
                 }
             } else {
                 // Hints are attached to predicates, so newJoinConditions should not be empty, but adding the check to be safe.
@@ -821,27 +887,31 @@
                     }
                 }
 
-                hjPlan = buildHashJoinPlan(leftJn, rightJn, hashJoinExpr, null);
-                if (!joinEnum.forceJoinOrderMode) {
-                    commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, hashJoinExpr, null);
+                hjPlan = buildHashJoinPlan(leftJn, rightJn, leftPlan, rightPlan, hashJoinExpr, null);
+                if (!joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel) {
+                    commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, rightPlan, leftPlan, hashJoinExpr, null);
                 }
-                bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, hashJoinExpr, null);
-                if (!joinEnum.forceJoinOrderMode) {
-                    commutativeBcastHjPlan = buildBroadcastHashJoinPlan(rightJn, leftJn, hashJoinExpr, null);
+                bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, leftPlan, rightPlan, hashJoinExpr, null);
+                if (!joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel) {
+                    commutativeBcastHjPlan =
+                            buildBroadcastHashJoinPlan(rightJn, leftJn, rightPlan, leftPlan, hashJoinExpr, null);
                 }
-                nljPlan = buildNLJoinPlan(leftJn, rightJn, nestedLoopJoinExpr);
-                if (!joinEnum.forceJoinOrderMode) {
-                    commutativeNljPlan = buildNLJoinPlan(rightJn, leftJn, nestedLoopJoinExpr);
+                nljPlan = buildNLJoinPlan(leftJn, rightJn, leftPlan, rightPlan, nestedLoopJoinExpr, null);
+                if (!joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel) {
+                    commutativeNljPlan =
+                            buildNLJoinPlan(rightJn, leftJn, rightPlan, leftPlan, nestedLoopJoinExpr, null);
                 }
-                cpPlan = buildCPJoinPlan(leftJn, rightJn, hashJoinExpr, nestedLoopJoinExpr);
-                if (!joinEnum.forceJoinOrderMode) {
-                    commutativeCpPlan = buildCPJoinPlan(rightJn, leftJn, hashJoinExpr, nestedLoopJoinExpr);
+                cpPlan = buildCPJoinPlan(leftJn, rightJn, leftPlan, rightPlan, hashJoinExpr, nestedLoopJoinExpr);
+                if (!joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel) {
+                    commutativeCpPlan =
+                            buildCPJoinPlan(rightJn, leftJn, rightPlan, leftPlan, hashJoinExpr, nestedLoopJoinExpr);
                 }
             }
         } else if (hintNLJoin != null) {
-            nljPlan = buildNLJoinPlan(leftJn, rightJn, nestedLoopJoinExpr);
-            if (!joinEnum.forceJoinOrderMode) {
-                commutativeNljPlan = buildNLJoinPlan(rightJn, leftJn, nestedLoopJoinExpr);
+            nljPlan = buildNLJoinPlan(leftJn, rightJn, leftPlan, rightPlan, nestedLoopJoinExpr, hintNLJoin);
+            if (!joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel) {
+                commutativeNljPlan =
+                        buildNLJoinPlan(rightJn, leftJn, rightPlan, leftPlan, nestedLoopJoinExpr, hintNLJoin);
             }
             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.
@@ -854,50 +924,84 @@
                                 ErrorCode.INAPPLICABLE_HINT, "index nested loop join", "ignored"));
                     }
                 }
-                hjPlan = buildHashJoinPlan(leftJn, rightJn, hashJoinExpr, null);
-                if (!joinEnum.forceJoinOrderMode) {
-                    commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, hashJoinExpr, null);
+                hjPlan = buildHashJoinPlan(leftJn, rightJn, leftPlan, rightPlan, hashJoinExpr, null);
+                if (!joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel) {
+                    commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, rightPlan, leftPlan, hashJoinExpr, null);
                 }
-                bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, hashJoinExpr, null);
-                if (!joinEnum.forceJoinOrderMode) {
-                    commutativeBcastHjPlan = buildBroadcastHashJoinPlan(rightJn, leftJn, hashJoinExpr, null);
+                bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, leftPlan, rightPlan, hashJoinExpr, null);
+                if (!joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel) {
+                    commutativeBcastHjPlan =
+                            buildBroadcastHashJoinPlan(rightJn, leftJn, rightPlan, leftPlan, hashJoinExpr, null);
                 }
-                cpPlan = buildCPJoinPlan(leftJn, rightJn, hashJoinExpr, nestedLoopJoinExpr);
-                if (!joinEnum.forceJoinOrderMode) {
-                    commutativeCpPlan = buildCPJoinPlan(rightJn, leftJn, hashJoinExpr, nestedLoopJoinExpr);
+                cpPlan = buildCPJoinPlan(leftJn, rightJn, leftPlan, rightPlan, hashJoinExpr, nestedLoopJoinExpr);
+                if (!joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel) {
+                    commutativeCpPlan =
+                            buildCPJoinPlan(rightJn, leftJn, rightPlan, leftPlan, hashJoinExpr, nestedLoopJoinExpr);
                 }
             }
         } else {
-            hjPlan = buildHashJoinPlan(leftJn, rightJn, hashJoinExpr, null);
-            if (!joinEnum.forceJoinOrderMode) {
-                commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, hashJoinExpr, null);
+            hjPlan = buildHashJoinPlan(leftJn, rightJn, leftPlan, rightPlan, hashJoinExpr, null);
+            if (!joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel) {
+                commutativeHjPlan = buildHashJoinPlan(rightJn, leftJn, rightPlan, leftPlan, hashJoinExpr, null);
             }
-            bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, hashJoinExpr, null);
-            if (!joinEnum.forceJoinOrderMode) {
-                commutativeBcastHjPlan = buildBroadcastHashJoinPlan(rightJn, leftJn, hashJoinExpr, null);
+            bcastHjPlan = buildBroadcastHashJoinPlan(leftJn, rightJn, leftPlan, rightPlan, hashJoinExpr, null);
+            if (!joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel) {
+                commutativeBcastHjPlan =
+                        buildBroadcastHashJoinPlan(rightJn, leftJn, rightPlan, leftPlan, hashJoinExpr, null);
             }
-            nljPlan = buildNLJoinPlan(leftJn, rightJn, nestedLoopJoinExpr);
-            if (!joinEnum.forceJoinOrderMode) {
-                commutativeNljPlan = buildNLJoinPlan(rightJn, leftJn, nestedLoopJoinExpr);
+            nljPlan = buildNLJoinPlan(leftJn, rightJn, leftPlan, rightPlan, nestedLoopJoinExpr, null);
+            if (!joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel) {
+                commutativeNljPlan = buildNLJoinPlan(rightJn, leftJn, rightPlan, leftPlan, nestedLoopJoinExpr, null);
             }
-            cpPlan = buildCPJoinPlan(leftJn, rightJn, hashJoinExpr, nestedLoopJoinExpr);
-            if (!joinEnum.forceJoinOrderMode) {
-                commutativeCpPlan = buildCPJoinPlan(rightJn, leftJn, hashJoinExpr, nestedLoopJoinExpr);
+            cpPlan = buildCPJoinPlan(leftJn, rightJn, leftPlan, rightPlan, hashJoinExpr, nestedLoopJoinExpr);
+            if (!joinEnum.forceJoinOrderMode || level <= joinEnum.cboFullEnumLevel) {
+                commutativeCpPlan =
+                        buildCPJoinPlan(rightJn, leftJn, rightPlan, leftPlan, 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);
+        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;
+        }
+    }
+
+    private PlanNode findCheapestPlan() {
+        List<PlanNode> allPlans = joinEnum.allPlans;
+        ICost cheapestCost = joinEnum.getCostHandle().maxCost();
+        PlanNode cheapestPlanNode = null;
+        boolean isCheapestPlanHinted = false;
+        boolean isPlanHinted;
+
+        for (int planIndex : this.planIndexesArray) {
+            PlanNode plan = allPlans.get(planIndex);
+            isPlanHinted = plan.joinHint != null || plan.indexHint;
+
+            if (isPlanHinted && !isCheapestPlanHinted) {
+                // The hinted plan wins!
+                cheapestPlanNode = plan;
+                cheapestCost = plan.totalCost;
+                isCheapestPlanHinted = true;
+            } else if (isPlanHinted || !isCheapestPlanHinted) {
+                // Either both plans are hinted, or both are non-hinted.
+                // Cost is the decider.
+                if (plan.totalCost.costLT(cheapestCost)) {
+                    cheapestPlanNode = plan;
+                    cheapestCost = plan.totalCost;
+                    isCheapestPlanHinted = isPlanHinted;
+                }
+            } else {
+                // this is the case where isPlanHinted == false AND isCheapestPlanHinted == true
+                // Nothing to do.
+            }
+        }
+        return cheapestPlanNode;
     }
 
     @Override
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
index 7e9c3ee..508e2ec 100644
--- 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
@@ -23,6 +23,7 @@
 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.expressions.IExpressionAnnotation;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 
@@ -41,6 +42,8 @@
     ICost leftExchangeCost;
     ICost rightExchangeCost;
     JoinMethod joinOp;
+    boolean indexHint;
+    IExpressionAnnotation joinHint;
     // Used to indicate which side to build for HJ and which side to broadcast for BHJ.
     HashJoinExpressionAnnotation.BuildSide side;
     ScanMethod scanOp;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/OperatorValueAccessPushdownVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/OperatorValueAccessPushdownVisitor.java
index 6739384..05bc161 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/OperatorValueAccessPushdownVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/OperatorValueAccessPushdownVisitor.java
@@ -74,6 +74,7 @@
 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.SwitchOperator;
 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;
@@ -362,6 +363,12 @@
     }
 
     @Override
+    public Void visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+        visitInputs(op);
+        return null;
+    }
+
+    @Override
     public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
         visitInputs(op);
         return null;
@@ -474,4 +481,4 @@
     private void visitInputs(ILogicalOperator op) throws AlgebricksException {
         visitInputs(op, null);
     }
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
index 4ac44b4..55165ed 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
@@ -80,6 +80,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 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.SwitchOperator;
 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;
@@ -570,6 +571,11 @@
     }
 
     @Override
+    public ILogicalOperator visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+        return visitSingleInputOperator(op);
+    }
+
+    @Override
     public ILogicalOperator visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
         return visitSingleInputOperator(op);
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
index d4b7853..cd476f9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
@@ -63,6 +63,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 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.SwitchOperator;
 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;
@@ -297,6 +298,11 @@
     }
 
     @Override
+    public ILogicalOperator visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+        return visitSingleInputOperator(op);
+    }
+
+    @Override
     public ILogicalOperator visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
         return visitSingleInputOperator(op);
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
index a0bb4b6..8bb7502 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
@@ -48,6 +48,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 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.SwitchOperator;
 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;
@@ -167,6 +168,11 @@
     }
 
     @Override
+    public Boolean visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+        return visitInputs(op);
+    }
+
+    @Override
     public Boolean visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
         return visitInputs(op);
     }
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..93f59908 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;
 
@@ -107,7 +106,7 @@
     }
 
     class Stats implements Serializable {
-        private static final long serialVersionUID = 5885273238208454611L;
+        private static final long serialVersionUID = 5885273238208454612L;
 
         public enum ProfileType {
             COUNTS("counts"),
@@ -134,9 +133,12 @@
         private long count;
         private long size;
         private long processedObjects;
+        private long queueWaitTime;
         private Profile profile;
         private ProfileType profileType;
         private long totalWarningsCount;
+        private long compileTime;
+        private double bufferCacheHitRatio;
 
         public long getCount() {
             return count;
@@ -154,6 +156,10 @@
             this.size = size;
         }
 
+        public long getQueueWaitTime() {
+            return queueWaitTime;
+        }
+
         public long getProcessedObjects() {
             return processedObjects;
         }
@@ -172,6 +178,10 @@
             }
         }
 
+        public void setQueueWaitTime(long queueWaitTime) {
+            this.queueWaitTime = queueWaitTime;
+        }
+
         public void setJobProfile(ObjectNode profile) {
             this.profile = new Profile(profile);
         }
@@ -187,10 +197,26 @@
         public void setProfileType(ProfileType profileType) {
             this.profileType = profileType;
         }
+
+        public void setCompileTime(long compileTime) {
+            this.compileTime = compileTime;
+        }
+
+        public long getCompileTime() {
+            return compileTime;
+        }
+
+        public void setBufferCacheHitRatio(double bufferCacheHitRatio) {
+            this.bufferCacheHitRatio = bufferCacheHitRatio;
+        }
+
+        public double getBufferCacheHitRatio() {
+            return bufferCacheHitRatio;
+        }
     }
 
     class Profile implements Serializable {
-        private static final long serialVersionUID = 4813321148252768375L;
+        private static final long serialVersionUID = 4813321148252768376L;
 
         private transient ObjectNode profile;
 
@@ -200,16 +226,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-algebra/src/main/java/org/apache/asterix/translator/ResultMetadata.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultMetadata.java
index 78f84ff..f47fce1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultMetadata.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultMetadata.java
@@ -27,16 +27,18 @@
 import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class ResultMetadata implements IResultMetadata {
-    private static final long serialVersionUID = 1905367559307369034L;
+    private static final long serialVersionUID = 1905367559307369035L;
 
     private final SessionConfig.OutputFormat format;
     private long jobDuration;
     private long processedObjects;
-    private ObjectNode profile;
     private long diskIoCount;
+    private double bufferCacheHitRatio;
+    private ObjectNode profile;
     private Set<Warning> warnings;
     private long totalWarningsCount;
     private transient List<Object> outputTypes;
+    private long queueWaitTimeInNanos;
 
     public ResultMetadata(SessionConfig.OutputFormat format) {
         this.format = format;
@@ -58,6 +60,14 @@
         this.jobDuration = jobDuration;
     }
 
+    public void setBufferCacheHitRatio(double bufferCacheHitRatio) {
+        this.bufferCacheHitRatio = bufferCacheHitRatio;
+    }
+
+    public double getBufferCacheHitRatio() {
+        return bufferCacheHitRatio;
+    }
+
     public void setWarnings(Set<Warning> warnings) {
         this.warnings = warnings;
     }
@@ -104,6 +114,14 @@
         return outputTypes;
     }
 
+    public long getQueueWaitTimeInNanos() {
+        return queueWaitTimeInNanos;
+    }
+
+    public void setQueueWaitTimeInNanos(long queueWaitTimeInNanos) {
+        this.queueWaitTimeInNanos = queueWaitTimeInNanos;
+    }
+
     @Override
     public String toString() {
         return "ResultMetadata{" + "format=" + format + ", jobDuration=" + jobDuration + ", processedObjects="
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 1d9c57b..af119d6 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
@@ -40,6 +40,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;
@@ -1044,6 +1045,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 2285d77..6c01245 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -155,6 +155,7 @@
           <usedDependencies combine.children="append">
             <usedDependency>org.apache.hadoop:hadoop-common</usedDependency>
             <usedDependency>org.apache.asterix:asterix-geo</usedDependency>
+            <usedDependency>org.apache.logging.log4j:log4j-jul</usedDependency>
           </usedDependencies>
           <ignoredUnusedDeclaredDependencies>
             <ignoredUnusedDeclaredDependency>org.apache.asterix:asterix-external-data:zip:*</ignoredUnusedDeclaredDependency>
@@ -389,7 +390,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 99678da..0cc2790 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
@@ -25,6 +25,7 @@
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -63,9 +64,7 @@
 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.ViewDecl;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.om.base.IAObject;
@@ -109,6 +108,7 @@
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 import org.apache.hyracks.api.exceptions.SourceLocation;
+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.resource.IClusterCapacity;
@@ -116,6 +116,7 @@
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectWriter;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 
 /**
  * Provides helper methods for compilation of a query into a JobSpec and submission
@@ -133,6 +134,7 @@
     private final IRuleSetFactory ruleSetFactory;
     private final Set<String> configurableParameterNames;
     private final ExecutionPlans executionPlans;
+    private PlanInfo lastPlan;
 
     public APIFramework(ILangCompilationProvider compilationProvider) {
         this.rewriterFactory = compilationProvider.getRewriterFactory();
@@ -141,6 +143,22 @@
         this.ruleSetFactory = compilationProvider.getRuleSetFactory();
         this.configurableParameterNames = compilationProvider.getCompilerOptions();
         executionPlans = new ExecutionPlans();
+        lastPlan = null;
+    }
+
+    private class PlanInfo {
+        ILogicalPlan plan;
+        Map<Object, String> log2Phys;
+        boolean printOptimizerEstimates;
+        SessionConfig.PlanFormat format;
+
+        public PlanInfo(ILogicalPlan plan, Map<Object, String> log2Phys, boolean printOptimizerEstimates,
+                SessionConfig.PlanFormat format) {
+            this.plan = plan;
+            this.log2Phys = log2Phys;
+            this.printOptimizerEstimates = printOptimizerEstimates;
+            this.format = format;
+        }
     }
 
     private static class OptimizationContextFactory implements IOptimizationContextFactory {
@@ -170,10 +188,9 @@
         }
     }
 
-    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;
         }
@@ -182,16 +199,14 @@
             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());
     }
 
     public JobSpecification compileQuery(IClusterInfoCollector clusterInfoCollector, MetadataProvider metadataProvider,
             Query query, int varCounter, String outputDatasetName, SessionOutput output,
             ICompiledDmlStatement statement, Map<VarIdentifier, IAObject> externalVars, IResponsePrinter printer,
-            IWarningCollector warningCollector, IRequestParameters requestParameters)
+            IWarningCollector warningCollector, IRequestParameters requestParameters, EnumSet<JobFlag> runtimeFlags)
             throws AlgebricksException, ACIDException {
 
         // establish facts
@@ -299,15 +314,7 @@
 
         boolean printSignature = isQuery && requestParameters != null && requestParameters.isPrintSignature();
 
-        if (isExplainOnly) {
-            printPlanAsResult(metadataProvider, output, printer, printSignature);
-            if (!conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN)) {
-                executionPlans.setOptimizedLogicalPlan(null);
-            }
-            return null;
-        }
-
-        if (printSignature) {
+        if (printSignature && !isExplainOnly) { //explainOnly adds the signature later
             printer.addResultPrinter(SignaturePrinter.newInstance(executionPlans));
         }
 
@@ -320,7 +327,7 @@
 
         JobEventListenerFactory jobEventListenerFactory =
                 new JobEventListenerFactory(txnId, metadataProvider.isWriteTransaction());
-        JobSpecification spec = compiler.createJob(ccAppContext, jobEventListenerFactory);
+        JobSpecification spec = compiler.createJob(ccAppContext, jobEventListenerFactory, runtimeFlags);
 
         if (isQuery) {
             if (!compiler.skipJobCapacityAssignment()) {
@@ -340,7 +347,12 @@
 
         if (conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN) || isExplainOnly) {
             if (isQuery || isLoad) {
-                generateOptimizedLogicalPlan(plan, output.config().getPlanFormat(), cboMode);
+                generateOptimizedLogicalPlan(plan, spec.getLogical2PhysicalMap(), output.config().getPlanFormat(),
+                        cboMode);
+                if (runtimeFlags.contains(JobFlag.PROFILE_RUNTIME)) {
+                    lastPlan =
+                            new PlanInfo(plan, spec.getLogical2PhysicalMap(), cboMode, output.config().getPlanFormat());
+                }
             }
         }
 
@@ -543,6 +555,26 @@
                 .setLogicalPlan(getPrettyPrintVisitor(format).printPlan(plan, printOptimizerEstimates).toString());
     }
 
+    private void generateOptimizedLogicalPlan(ILogicalPlan plan, Map<Object, String> log2phys,
+            SessionConfig.PlanFormat format, boolean printOptimizerEstimates) throws AlgebricksException {
+        executionPlans.setOptimizedLogicalPlan(
+                getPrettyPrintVisitor(format).printPlan(plan, log2phys, printOptimizerEstimates).toString());
+    }
+
+    public void generateOptimizedLogicalPlanWithProfile(ObjectNode profile) throws HyracksDataException {
+        /*TODO(ian): we call this and overwrite the non-annotated plan, but there should be some way to skip initial
+                     plan printing if both profiling and plan printing are requested. */
+        try {
+            if (lastPlan != null) {
+                executionPlans.setOptimizedLogicalPlan(getPrettyPrintVisitor(lastPlan.format)
+                        .printPlan(lastPlan.plan, lastPlan.log2Phys, lastPlan.printOptimizerEstimates, profile)
+                        .toString());
+            }
+        } catch (AlgebricksException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
     private void generateOptimizedLogicalPlan(ILogicalPlan plan, SessionConfig.PlanFormat format,
             boolean printOptimizerEstimates) throws AlgebricksException {
         executionPlans.setOptimizedLogicalPlan(
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 9fa479c..a5e13c1 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
@@ -205,6 +205,9 @@
         stats.setJobProfile(responseStats.getJobProfile());
         stats.setProcessedObjects(responseStats.getProcessedObjects());
         stats.updateTotalWarningsCount(responseStats.getTotalWarningsCount());
+        stats.setCompileTime(responseStats.getCompileTime());
+        stats.setQueueWaitTime(responseStats.getQueueWaitTime());
+        stats.setBufferCacheHitRatio(responseStats.getBufferCacheHitRatio());
     }
 
     private static void updatePropertiesFromCC(IStatementExecutor.StatementProperties statementProperties,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java
index 3d0f7fc..47685d8 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java
@@ -81,7 +81,7 @@
     protected final IReceptionist receptionist;
 
     protected Path workingDir;
-    protected String sysAuthHeader;
+    private String sysAuthHeader;
     private ILibraryManager libraryManager;
     private int timeout;
 
@@ -250,7 +250,7 @@
                 }
                 URI downloadURI = createDownloadURI(libraryTempFile);
                 doCreate(dvAndName.getFirst(), dvAndName.getSecond(), language,
-                        ExternalLibraryUtils.digestToHexString(digest), downloadURI, true, sysAuthHeader,
+                        ExternalLibraryUtils.digestToHexString(digest), downloadURI, true, getSysAuthHeader(),
                         requestReference, request);
             } else if (op == LibraryOperation.DELETE) {
                 //DELETE semantics imply ifExists
@@ -281,6 +281,10 @@
         }
     }
 
+    protected String getSysAuthHeader() {
+        return sysAuthHeader;
+    }
+
     private void writeException(Exception e, IServletResponse response) {
         response.setStatus(toHttpErrorStatus(e));
         PrintWriter responseWriter = response.writer();
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 bfebfd6..ad2a5b5 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
@@ -102,7 +102,8 @@
                 printer.printResults();
                 ResponseMetrics metrics = ResponseMetrics.of(System.nanoTime() - elapsedStart,
                         metadata.getJobDuration(), stats.getCount(), stats.getSize(), metadata.getProcessedObjects(), 0,
-                        metadata.getTotalWarningsCount());
+                        metadata.getTotalWarningsCount(), stats.getCompileTime(), stats.getQueueWaitTime(),
+                        stats.getBufferCacheHitRatio());
                 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 9a8c0d5..f84e45a 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
@@ -361,7 +361,8 @@
         }
         final ResponseMetrics metrics =
                 ResponseMetrics.of(System.nanoTime() - elapsedStart, executionState.duration(), stats.getCount(),
-                        stats.getSize(), stats.getProcessedObjects(), errorCount, stats.getTotalWarningsCount());
+                        stats.getSize(), stats.getProcessedObjects(), errorCount, stats.getTotalWarningsCount(),
+                        stats.getCompileTime(), stats.getQueueWaitTime(), stats.getBufferCacheHitRatio());
         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/RebalanceApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
index 58a37bd..021c3fb 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
@@ -261,7 +261,7 @@
         try {
             ActiveNotificationHandler activeNotificationHandler =
                     (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
-            activeNotificationHandler.suspend(metadataProvider);
+            activeNotificationHandler.suspend(metadataProvider, "rebalance api");
             try {
                 IMetadataLockManager lockManager = appCtx.getMetadataLockManager();
                 lockManager.acquireDatasetExclusiveModificationLock(metadataProvider.getLocks(), dataverseName,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
index 626b938..0240518 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
@@ -138,7 +138,7 @@
     }
 
     protected synchronized void setState(ActivityState newState) {
-        LOGGER.log(level, "State of {} is being set to {} from {}", getEntityId(), newState, state);
+        LOGGER.log(level, "state of {} is being set from {} to {}", getEntityId(), state, newState);
         this.prevState = state;
         this.state = newState;
         if (newState == ActivityState.STARTING || newState == ActivityState.RECOVERING
@@ -153,9 +153,8 @@
     @Override
     public synchronized void notify(ActiveEvent event) {
         try {
-            if (LOGGER.isEnabled(level)) {
-                LOGGER.log(level, "EventListener is notified.");
-            }
+            LOGGER.debug("CC handling event {}; state={}, prev state={}, suspended={}", event, state, prevState,
+                    suspended);
             ActiveEvent.Kind eventKind = event.getEventKind();
             switch (eventKind) {
                 case JOB_CREATED:
@@ -194,26 +193,21 @@
 
     @SuppressWarnings("unchecked")
     protected void finish(ActiveEvent event) throws HyracksDataException {
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "Active job {} finished", jobId);
-        }
         JobId lastJobId = jobId;
+        Pair<JobStatus, List<Exception>> status = (Pair<JobStatus, List<Exception>>) event.getEventObject();
         if (numRegistered != numDeRegistered) {
             LOGGER.log(Level.WARN,
-                    "Active job {} finished with reported runtime registrations = {} and deregistrations = {}", jobId,
-                    numRegistered, numDeRegistered);
+                    "ingestion job {} finished with status={}, reported runtime registrations={}, deregistrations={}",
+                    jobId, status, numRegistered, numDeRegistered);
         }
         jobId = null;
-        Pair<JobStatus, List<Exception>> status = (Pair<JobStatus, List<Exception>>) event.getEventObject();
         JobStatus jobStatus = status.getLeft();
         List<Exception> exceptions = status.getRight();
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "Active job {} finished with status {}", lastJobId, jobStatus);
-        }
+        LOGGER.debug("ingestion job {} finished with status {}", lastJobId, jobStatus);
         if (!jobSuccessfullyTerminated(jobStatus)) {
             jobFailure = exceptions.isEmpty() ? new RuntimeDataException(ErrorCode.UNREPORTED_TASK_FAILURE_EXCEPTION)
                     : exceptions.get(0);
-            LOGGER.error("Active Job {} failed", lastJobId, jobFailure);
+            LOGGER.error("ingestion job {} failed", lastJobId, jobFailure);
             setState((state == ActivityState.STOPPING || state == ActivityState.CANCELLING) ? ActivityState.STOPPED
                     : ActivityState.TEMPORARILY_FAILED);
             if (prevState == ActivityState.RUNNING) {
@@ -371,16 +365,14 @@
 
     @Override
     public synchronized void recover() {
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "Recover is called on {}", entityId);
-        }
         if (retryPolicyFactory == NoRetryPolicyFactory.INSTANCE) {
-            LOGGER.log(level, "But it has no recovery policy, so it is set to permanent failure");
+            LOGGER.debug("recover is called on {} w/o recovery policy; setting to permanent failure", entityId);
             setState(ActivityState.STOPPED);
         } else {
+            LOGGER.debug("recover is called on {}", entityId);
             ExecutorService executor = appCtx.getServiceContext().getControllerService().getExecutor();
             setState(ActivityState.TEMPORARILY_FAILED);
-            LOGGER.log(level, "Recovery task has been submitted");
+            LOGGER.debug("recovery task has been submitted");
             rt = createRecoveryTask();
             executor.submit(rt.recover());
         }
@@ -479,15 +471,11 @@
         // Note: once we start sending stop messages, we can't go back until the entity is stopped
         final String nameBefore = Thread.currentThread().getName();
         try {
-            Thread.currentThread().setName(nameBefore + " : WaitForCompletionForJobId: " + jobId);
+            Thread.currentThread().setName(nameBefore + " : wait-for-ingestion-completion: " + jobId);
             sendStopMessages(metadataProvider, timeout, unit);
-            if (LOGGER.isDebugEnabled()) {
-                LOGGER.debug("Waiting for its state to become " + waitFor);
-            }
+            LOGGER.debug("waiting for {} to become {}", jobId, waitFor);
             subscriber.sync();
-            if (LOGGER.isDebugEnabled()) {
-                LOGGER.debug("Disconnect has been completed " + waitFor);
-            }
+            LOGGER.debug("disconnect has been completed {}", waitFor);
         } catch (InterruptedException ie) {
             forceStop(subscriber, ie);
             Thread.currentThread().interrupt();
@@ -517,12 +505,9 @@
             LOGGER.log(Level.INFO, "Sending stop messages to " + runtimeLocations);
         }
         for (String location : runtimeLocations.getLocations()) {
-            if (LOGGER.isInfoEnabled()) {
-                LOGGER.log(Level.INFO, "Sending to " + location);
-            }
             ActiveRuntimeId runtimeId = getActiveRuntimeId(partition++);
             messageBroker.sendApplicationMessageToNC(new ActiveManagerMessage(ActiveManagerMessage.Kind.STOP_ACTIVITY,
-                    runtimeId, new StopRuntimeParameters(timeout, unit)), location);
+                    runtimeId, new StopRuntimeParameters(timeout, unit), ""), location);
         }
     }
 
@@ -736,7 +721,7 @@
 
     @Override
     public String toString() {
-        return "{\"class\":\"" + getClass().getSimpleName() + "\"," + "\"entityId\":\"" + entityId + "\","
-                + "\"state\":\"" + state + "\"" + "}";
+        return "{\"class\":\"" + getClass().getSimpleName() + "\", \"entityId\":\"" + entityId + "\", \"state\":\""
+                + state + "\", \"prev state\":\"" + prevState + "\", \"suspended\":" + suspended + "}";
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
index 3c277d5..8821c67 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
@@ -72,19 +72,17 @@
         EntityId entityId = jobId2EntityId.get(jobId);
         if (entityId != null) {
             IActiveEntityEventsListener listener = entityEventListeners.get(entityId);
-            if (LOGGER.isEnabled(level)) {
-                LOGGER.log(level, "Next event is {} for job {}", eventKind, jobId);
-            }
             if (eventKind == Kind.JOB_FINISHED) {
-                LOGGER.log(level, "Removing job {}", jobId);
+                LOGGER.debug("removing ingestion job {}", jobId);
                 jobId2EntityId.remove(jobId);
             }
             if (listener != null) {
-                LOGGER.log(level, "Notifying the listener");
                 listener.notify(event);
+            } else {
+                LOGGER.debug("listener not found for entity {} on event={}", entityId, event);
             }
         } else {
-            LOGGER.log(Level.ERROR, "Entity not found for event {} for job {}", eventKind, jobId);
+            LOGGER.error("entity not found for event {}", event);
         }
     }
 
@@ -92,45 +90,30 @@
 
     @Override
     public void notifyJobCreation(JobId jobId, JobSpecification jobSpecification) throws HyracksDataException {
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "notifyJobCreation was called for job {}", jobId);
-        }
         Object property = jobSpecification.getProperty(ACTIVE_ENTITY_PROPERTY_NAME);
         if (!(property instanceof EntityId)) {
-            if (LOGGER.isEnabled(level)) {
-                LOGGER.log(level, "Job {} is not of type active job. property found to be {}", jobId, property);
+            if (property != null) {
+                LOGGER.debug("{} is not an active job. job property={}", jobId, property);
             }
             return;
         }
+        LOGGER.debug("notified of ingestion job creation {}", jobId);
         EntityId entityId = (EntityId) property;
         monitorJob(jobId, entityId);
-        boolean found = jobId2EntityId.get(jobId) != null;
-        LOGGER.log(level, "Job {} was found to be {}", jobId, (found ? "Active" : "Inactive"));
         add(new ActiveEvent(jobId, Kind.JOB_CREATED, entityId, jobSpecification));
     }
 
     private synchronized void monitorJob(JobId jobId, EntityId entityId) {
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "monitorJob was called for job {}", jobId);
-        }
-        boolean found = jobId2EntityId.get(jobId) != null;
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "Job {} was found to be {}", jobId, (found ? "Active" : "Inactive"));
-        }
+        boolean found = jobId2EntityId.containsKey(jobId);
+        LOGGER.debug("{} is {}", jobId, (found ? "active" : "inactive"));
         if (entityEventListeners.containsKey(entityId)) {
-            if (jobId2EntityId.containsKey(jobId)) {
-                if (LOGGER.isErrorEnabled()) {
-                    LOGGER.error("Job {} is already being monitored", jobId);
-                }
+            if (found) {
+                LOGGER.error("{} is already being monitored", jobId);
                 return;
             }
-            if (LOGGER.isEnabled(level)) {
-                LOGGER.log(level, "Monitoring started for job {}", jobId);
-            }
+            LOGGER.debug("monitoring started for {}", jobId);
         } else {
-            if (LOGGER.isEnabled(level)) {
-                LOGGER.info("No listener was found for the entity {} for job {}", entityId, jobId);
-            }
+            LOGGER.debug("no listener found for entity {}; {}", entityId, jobId);
         }
         jobId2EntityId.put(jobId, entityId);
     }
@@ -141,22 +124,18 @@
         if (entityId != null) {
             add(new ActiveEvent(jobId, Kind.JOB_STARTED, entityId, null));
         }
+        // else must be non-active job, e.g. a job for a query
     }
 
     @Override
     public synchronized void notifyJobFinish(JobId jobId, JobStatus jobStatus, List<Exception> exceptions)
             throws HyracksException {
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "Getting notified of job finish for job {}", jobId);
-        }
         EntityId entityId = jobId2EntityId.get(jobId);
         if (entityId != null) {
+            LOGGER.debug("notified of ingestion job finish {}", jobId);
             add(new ActiveEvent(jobId, Kind.JOB_FINISHED, entityId, Pair.of(jobStatus, exceptions)));
-        } else {
-            if (LOGGER.isEnabled(level)) {
-                LOGGER.log(level, "No need to notify JOB_FINISHED for job {}", jobId);
-            }
         }
+        // else must be non-active job, e.g. a job for a query
     }
 
     // *** IActiveNotificationHandler
@@ -169,13 +148,6 @@
 
     @Override
     public IActiveEntityEventsListener getListener(EntityId entityId) {
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "getActiveEntityListener was called with entity {}", entityId);
-        }
-        IActiveEntityEventsListener listener = entityEventListeners.get(entityId);
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "Listener found: {}", listener);
-        }
         return entityEventListeners.get(entityId);
     }
 
@@ -197,9 +169,7 @@
         if (suspended) {
             throw new RuntimeDataException(ErrorCode.ACTIVE_NOTIFICATION_HANDLER_IS_SUSPENDED);
         }
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "registerListener was called for the entity {}", listener.getEntityId());
-        }
+        LOGGER.debug("register listener for entity {}, state={}", listener.getEntityId(), listener.getState());
         if (entityEventListeners.containsKey(listener.getEntityId())) {
             throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_IS_ALREADY_REGISTERED, listener.getEntityId());
         }
@@ -211,9 +181,7 @@
         if (suspended) {
             throw new RuntimeDataException(ErrorCode.ACTIVE_NOTIFICATION_HANDLER_IS_SUSPENDED);
         }
-        if (LOGGER.isEnabled(level)) {
-            LOGGER.log(level, "unregisterListener was called for the entity {}", listener.getEntityId());
-        }
+        LOGGER.debug("unregister listener for entity {}, state={}", listener.getEntityId(), listener.getState());
         IActiveEntityEventsListener registeredListener = entityEventListeners.remove(listener.getEntityId());
         if (registeredListener == null) {
             throw new RuntimeDataException(ErrorCode.ACTIVE_ENTITY_LISTENER_IS_NOT_REGISTERED, listener.getEntityId());
@@ -229,20 +197,19 @@
         LOGGER.info("Starting active recovery");
         for (IActiveEntityEventsListener listener : getEventListeners()) {
             synchronized (listener) {
-                if (LOGGER.isEnabled(level)) {
-                    LOGGER.log(level, "Entity {} is {}", listener.getEntityId(), listener.getState());
-                }
+                LOGGER.debug("entity {} is {}, active={}, suspended={}", listener.getEntityId(), listener.getState(),
+                        listener.isActive(), listener.isSuspended());
                 listener.notifyAll();
             }
         }
     }
 
-    public void suspend(MetadataProvider mdProvider) throws HyracksDataException {
+    public void suspend(MetadataProvider mdProvider, String reason) throws HyracksDataException {
         synchronized (this) {
             if (suspended) {
                 throw new RuntimeDataException(ErrorCode.ACTIVE_EVENT_HANDLER_ALREADY_SUSPENDED);
             }
-            LOGGER.log(level, "Suspending active events handler");
+            LOGGER.debug("suspending active events handler. reason {}", reason);
             suspended = true;
         }
         Collection<IActiveEntityEventsListener> registeredListeners = entityEventListeners.values();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/ActiveRequestsDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/ActiveRequestsDatasource.java
index 24a127a..5239d01 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/ActiveRequestsDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/ActiveRequestsDatasource.java
@@ -43,4 +43,9 @@
                 AlgebricksAbsolutePartitionConstraint.randomLocation(locations.getLocations());
         return new ClientRequestsFunction(randomLocation, ClientRequestsRequest.RequestType.RUNNING);
     }
+
+    @Override
+    public boolean skipJobCapacityAssignment() {
+        return true;
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/CompletedRequestsDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/CompletedRequestsDatasource.java
index 0bfbff1..f02af21 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/CompletedRequestsDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/CompletedRequestsDatasource.java
@@ -43,4 +43,9 @@
                 AlgebricksAbsolutePartitionConstraint.randomLocation(locations.getLocations());
         return new ClientRequestsFunction(randomLocation, ClientRequestsRequest.RequestType.COMPLETED);
     }
+
+    @Override
+    public boolean skipJobCapacityAssignment() {
+        return true;
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
index cf2b891..52cd17c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/QueryIndexDatasource.java
@@ -50,6 +50,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 QueryIndexDatasource extends FunctionDataSource {
 
@@ -105,7 +106,8 @@
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
             IProjectionInfo<?> projectionInfo) throws AlgebricksException {
         return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false, null, ds, indexName,
-                null, null, true, true, false, null, null, null, tupleFilterFactory, outputLimit, false, false);
+                null, null, true, true, false, null, null, null, tupleFilterFactory, outputLimit, false, false,
+                DefaultTupleProjectorFactory.INSTANCE);
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
index 65d1039..6570041 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
@@ -69,10 +69,8 @@
                 }
             }
         }
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Sending CancelQueryResponse to {}. requestId:{}, uuid:{}, contextId:{}, status:{}", nodeId,
-                    requestId, uuid, contextId, status);
-        }
+        LOGGER.debug("sending CancelQueryResponse to {}. reqId:{}, uuid:{}, contextId:{}, status:{}", nodeId, requestId,
+                uuid, contextId, status);
         CancelQueryResponse response = new CancelQueryResponse(reqId, status);
         CCMessageBroker messageBroker = (CCMessageBroker) appCtx.getServiceContext().getMessageBroker();
         try {
@@ -82,4 +80,9 @@
         }
     }
 
+    @Override
+    public String toString() {
+        return "CancelQueryRequest{from='" + nodeId + "', reqId=" + reqId + ", uuid='" + uuid + "', contextId='"
+                + contextId + "'}";
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryResponse.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryResponse.java
index d65ae31..68d3430 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryResponse.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryResponse.java
@@ -49,4 +49,8 @@
         return status;
     }
 
+    @Override
+    public String toString() {
+        return "CancelQueryResponse{reqId=" + reqId + ", status=" + status + '}';
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
index e314177..03ba4a9 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
@@ -150,7 +150,8 @@
         ILangCompilationProvider compilationProvider = ccExtMgr.getCompilationProvider(lang);
         IStorageComponentProvider storageComponentProvider = ccAppCtx.getStorageComponentProvider();
         IStatementExecutorFactory statementExecutorFactory = ccApp.getStatementExecutorFactory();
-        ExecuteStatementResponseMessage responseMsg = new ExecuteStatementResponseMessage(requestMessageId);
+        ExecuteStatementResponseMessage responseMsg =
+                new ExecuteStatementResponseMessage(requestMessageId, clientContextID, requestReference.getUuid());
         final IStatementExecutor.StatementProperties statementProperties = new IStatementExecutor.StatementProperties();
         responseMsg.setStatementProperties(statementProperties);
         try {
@@ -230,9 +231,10 @@
         return null;
     }
 
-    protected static void sendRejection(Exception reason, CCMessageBroker messageBroker, long requestMessageId,
+    protected void sendRejection(Exception reason, CCMessageBroker messageBroker, long requestMessageId,
             String requestNodeId) {
-        ExecuteStatementResponseMessage responseMsg = new ExecuteStatementResponseMessage(requestMessageId);
+        ExecuteStatementResponseMessage responseMsg =
+                new ExecuteStatementResponseMessage(requestMessageId, clientContextID, requestReference.getUuid());
         responseMsg.setError(reason);
         try {
             messageBroker.sendApplicationMessageToNC(responseMsg, requestNodeId);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementResponseMessage.java
index 2cdede1..eaadebe 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementResponseMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementResponseMessage.java
@@ -31,9 +31,12 @@
 import org.apache.hyracks.api.exceptions.Warning;
 
 public final class ExecuteStatementResponseMessage implements INcAddressedMessage {
-    private static final long serialVersionUID = 1L;
+
+    private static final long serialVersionUID = 2L;
 
     private final long requestMessageId;
+    private final String clientContextID;
+    private final String uuid;
 
     private String result;
 
@@ -49,8 +52,10 @@
 
     private Collection<Warning> warnings;
 
-    public ExecuteStatementResponseMessage(long requestMessageId) {
+    public ExecuteStatementResponseMessage(long requestMessageId, String clientContextID, String uuid) {
         this.requestMessageId = requestMessageId;
+        this.clientContextID = clientContextID;
+        this.uuid = uuid;
     }
 
     @Override
@@ -120,7 +125,7 @@
 
     @Override
     public String toString() {
-        return String.format("%s(id=%s): %d characters", getClass().getSimpleName(), requestMessageId,
-                result != null ? result.length() : 0);
+        return String.format("%s(id=%s, uuid=%s, clientContextId=%s): %d characters", getClass().getSimpleName(),
+                requestMessageId, uuid, clientContextID, result != null ? result.length() : 0);
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksResponseMessage.java
index bee5ff9..9d378f5 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksResponseMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksResponseMessage.java
@@ -100,4 +100,9 @@
     public MessageType getType() {
         return MessageType.REGISTRATION_TASKS_RESPONSE;
     }
+
+    @Override
+    public String toString() {
+        return "RegistrationTasksResponseMessage{from='" + nodeId + '}';
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java
index 25e51bb..5d74c13 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java
@@ -26,6 +26,7 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 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.PhysicalOperatorTag;
@@ -59,6 +60,7 @@
 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.SwitchOperator;
 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;
@@ -197,6 +199,12 @@
     }
 
     @Override
+    public Void visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+        // TODO (GLENN): Implement this logic
+        throw new NotImplementedException();
+    }
+
+    @Override
     public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
         visit(op);
         return null;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
index 024a13e..af383c3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
@@ -24,6 +24,7 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
@@ -57,6 +58,7 @@
 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.SwitchOperator;
 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;
@@ -193,6 +195,12 @@
     }
 
     @Override
+    public Void visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+        // TODO (GLENN): Implement this logic
+        throw new NotImplementedException();
+    }
+
+    @Override
     public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
         visitInternal(op, true);
         return null;
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 201a470..7ac1431 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
@@ -71,6 +71,8 @@
     private void aggregateJobStats(JobId jobId, ResultMetadata metadata) {
         long processedObjects = 0;
         long aggregateTotalWarningsCount = 0;
+        long pagesRead = 0;
+        long nonPagedReads = 0;
         Set<Warning> AggregateWarnings = new HashSet<>();
         IJobManager jobManager =
                 ((ClusterControllerService) appCtx.getServiceContext().getControllerService()).getJobManager();
@@ -82,7 +84,9 @@
             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();
+                    pagesRead += tp.getStatsCollector().getAggregatedStats().getPageReads().get();
+                    nonPagedReads += tp.getStatsCollector().getAggregatedStats().coldReadCounter().get();
                     aggregateTotalWarningsCount += tp.getTotalWarningsCount();
                     Set<Warning> taskWarnings = tp.getWarnings();
                     if (AggregateWarnings.size() < maxWarnings && !taskWarnings.isEmpty()) {
@@ -93,8 +97,10 @@
                     }
                 }
             }
+            metadata.setQueueWaitTimeInNanos(run.getJobProfile().getQueueWaitTimeInNanos());
         }
         metadata.setProcessedObjects(processedObjects);
+        metadata.setBufferCacheHitRatio(pagesRead > 0 ? (pagesRead - nonPagedReads) / (double) pagesRead : Double.NaN);
         metadata.setWarnings(AggregateWarnings);
         metadata.setTotalWarningsCount(aggregateTotalWarningsCount);
         if (run != null && run.getFlags() != null && run.getFlags().contains(JobFlag.PROFILE_RUNTIME)) {
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..5f70937 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,16 @@
     private long errorCount;
     private long warnCount;
     private long diskIoCount;
+    private long compileTime;
+    private long queueWaitTime;
+    private double bufferCacheHitRatio;
 
     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, long queueWaitTime,
+            double bufferCacheHitRatio) {
         ResponseMetrics metrics = new ResponseMetrics();
         metrics.elapsedTime = elapsedTime;
         metrics.executionTime = executionTime;
@@ -42,6 +46,9 @@
         metrics.processedObjects = processedObjects;
         metrics.errorCount = errorCount;
         metrics.warnCount = warnCount;
+        metrics.compileTime = compileTime;
+        metrics.queueWaitTime = queueWaitTime;
+        metrics.bufferCacheHitRatio = bufferCacheHitRatio;
         return metrics;
     }
 
@@ -72,4 +79,16 @@
     public long getWarnCount() {
         return warnCount;
     }
+
+    public long getCompileTime() {
+        return compileTime;
+    }
+
+    public long getQueueWaitTime() {
+        return queueWaitTime;
+    }
+
+    public double getBufferCacheHitRatio() {
+        return bufferCacheHitRatio;
+    }
 }
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..93fb6d4 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,11 +32,14 @@
     public enum Metrics {
         ELAPSED_TIME("elapsedTime"),
         EXECUTION_TIME("executionTime"),
+        COMPILE_TIME("compileTime"),
+        QUEUE_WAIT_TIME("queueWaitTime"),
         RESULT_COUNT("resultCount"),
         RESULT_SIZE("resultSize"),
         ERROR_COUNT("errorCount"),
         PROCESSED_OBJECTS_COUNT("processedObjects"),
-        WARNING_COUNT("warningCount");
+        WARNING_COUNT("warningCount"),
+        BUFFERCACHE_HIT_RATIO("bufferCacheHitRatio");
 
         private final String str;
 
@@ -71,15 +74,27 @@
         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.QUEUE_WAIT_TIME.str(),
+                Duration.formatNanos(metrics.getQueueWaitTime(), 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);
         pw.print("\n\t");
         final boolean hasErrors = metrics.getErrorCount() > 0;
         final boolean hasWarnings = metrics.getWarnCount() > 0;
+        final boolean usedCache = !(Double.isNaN(metrics.getBufferCacheHitRatio()));
         ResultUtil.printField(pw, Metrics.PROCESSED_OBJECTS_COUNT.str(), metrics.getProcessedObjects(),
-                hasWarnings || hasErrors);
+                usedCache || hasWarnings || hasErrors);
         pw.print("\n");
+        if (usedCache) {
+            pw.print("\t");
+            String pctValue = String.format("%.2f%%", metrics.getBufferCacheHitRatio() * 100);
+            ResultUtil.printField(pw, Metrics.BUFFERCACHE_HIT_RATIO.str(), pctValue, hasWarnings || hasErrors);
+            pw.print("\n");
+        }
         if (hasWarnings) {
             pw.print("\t");
             ResultUtil.printField(pw, Metrics.WARNING_COUNT.str(), metrics.getWarnCount(), hasErrors);
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 6b40cbf..0db8dff 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,6 +110,7 @@
 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;
@@ -440,6 +441,9 @@
                         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:
@@ -450,6 +454,9 @@
                                     || 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;
@@ -561,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;
@@ -2754,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);
@@ -3029,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);
@@ -3645,7 +3663,7 @@
                     loadStmt.getDatasetName(), loadStmt.getAdapter(), properties, loadStmt.dataIsAlreadySorted());
             cls.setSourceLocation(stmt.getSourceLocation());
             JobSpecification spec = apiFramework.compileQuery(hcc, metadataProvider, null, 0, null, sessionOutput, cls,
-                    null, responsePrinter, warningCollector, null);
+                    null, responsePrinter, warningCollector, null, jobFlags);
             afterCompile();
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
@@ -3768,16 +3786,18 @@
         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,
                 rewrittenResult.second, stmt == null ? null : stmt.getDatasetName(), sessionOutput, stmt, externalVars,
-                responsePrinter, warningCollector, requestParameters);
+                responsePrinter, warningCollector, requestParameters, jobFlags);
     }
 
-    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();
@@ -3785,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());
@@ -3813,7 +3835,7 @@
         // transaction)
         return apiFramework.compileQuery(clusterInfoCollector, metadataProvider, rewrittenInsertUpsert.getQuery(),
                 rewrittenResult.second, datasetName, sessionOutput, clfrqs, externalVars, responsePrinter,
-                warningCollector, null);
+                warningCollector, null, jobFlags);
     }
 
     protected void handleCreateFeedStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
@@ -4681,6 +4703,7 @@
             }
         };
         final IStatementCompiler compiler = () -> {
+            long compileStart = System.nanoTime();
             MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
             boolean bActiveTxn = true;
             metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -4691,6 +4714,7 @@
                 stats.updateTotalWarningsCount(warningCollector.getTotalWarningsCount());
                 afterCompile();
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                stats.setCompileTime(System.nanoTime() - compileStart);
                 bActiveTxn = false;
                 return query.isExplain() || isCompileOnly() ? null : jobSpec;
             } catch (Exception e) {
@@ -4754,8 +4778,11 @@
                 (org.apache.asterix.translator.ResultMetadata) controllerService.getResultDirectoryService()
                         .getResultMetadata(jobId, rsId);
         stats.setProcessedObjects(resultMetadata.getProcessedObjects());
+        stats.setQueueWaitTime(resultMetadata.getQueueWaitTimeInNanos());
+        stats.setBufferCacheHitRatio(resultMetadata.getBufferCacheHitRatio());
         if (jobFlags.contains(JobFlag.PROFILE_RUNTIME)) {
             stats.setJobProfile(resultMetadata.getJobProfile());
+            apiFramework.generateOptimizedLogicalPlanWithProfile(resultMetadata.getJobProfile());
         }
         stats.updateTotalWarningsCount(resultMetadata.getTotalWarningsCount());
         WarningUtil.mergeWarnings(resultMetadata.getWarnings(), warningCollector);
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 d835664..d8b47b8 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-cloud-storage.conf b/asterixdb/asterix-app/src/main/resources/cc-cloud-storage.conf
index a61ac87..be9bd54 100644
--- a/asterixdb/asterix-app/src/main/resources/cc-cloud-storage.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc-cloud-storage.conf
@@ -1,5 +1,3 @@
-<<<<<<< HEAD   (5f076a [ASTERIXDB-3221][REPL] Use IO scheduler for replication ops)
-=======
 ; Licensed to the Apache Software Foundation (ASF) under one
 ; or more contributor license agreements.  See the NOTICE file
 ; distributed with this work for additional information
@@ -68,4 +66,3 @@
 messaging.frame.count=512
 cloud.deployment=true
 storage.buffercache.pagesize=32KB
->>>>>>> CHANGE (337517 [NO ISSUE][OTH] Move page size storage config to common conf)
diff --git a/asterixdb/asterix-app/src/main/resources/cc_static_partitioning.conf b/asterixdb/asterix-app/src/main/resources/cc_static_partitioning.conf
index e3e779e..3923222 100644
--- a/asterixdb/asterix-app/src/main/resources/cc_static_partitioning.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc_static_partitioning.conf
@@ -1,5 +1,3 @@
-<<<<<<< HEAD   (5f076a [ASTERIXDB-3221][REPL] Use IO scheduler for replication ops)
-=======
 ; Licensed to the Apache Software Foundation (ASF) under one
 ; or more contributor license agreements.  See the NOTICE file
 ; distributed with this work for additional information
@@ -68,4 +66,3 @@
 messaging.frame.count=512
 storage.buffercache.pagesize=32KB
 storage.partitioning=static
->>>>>>> CHANGE (337517 [NO ISSUE][OTH] Move page size storage config to common conf)
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 34f4e40..1e096ac 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;
@@ -133,13 +134,13 @@
         cc = createCC(ccApplication, ccConfig);
 
         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();
@@ -314,7 +315,7 @@
 
         stopCC(false);
 
-        if (deleteOldInstanceData) {
+        if (deleteOldInstanceData && nodeNames != null) {
             deleteTransactionLogs();
             removeTestStorageFiles();
             deleteCCFiles();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/common/TestDataUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/common/TestDataUtil.java
index 06380fe..3449ee3 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/common/TestDataUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/common/TestDataUtil.java
@@ -162,7 +162,7 @@
         try {
             ActiveNotificationHandler activeNotificationHandler =
                     (ActiveNotificationHandler) ccAppCtx.getActiveNotificationHandler();
-            activeNotificationHandler.suspend(metadataProvider);
+            activeNotificationHandler.suspend(metadataProvider, "");
             try {
                 IMetadataLockManager lockManager = ccAppCtx.getMetadataLockManager();
                 lockManager.acquireDatasetExclusiveModificationLock(metadataProvider.getLocks(), dataverseName,
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 cb123bf..126edb1 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
@@ -146,7 +146,7 @@
         Assert.assertTrue(requestedStats.contains("N/A"));
         // Fake partition message and notify eventListener
         ActivePartitionMessage partitionMessage =
-                new ActivePartitionMessage(activeRuntimeId, jobId, Event.RUNTIME_REGISTERED, null);
+                new ActivePartitionMessage(activeRuntimeId, jobId, Event.RUNTIME_REGISTERED, null, "");
         partitionMessage.handle(appCtx);
         start.sync();
         if (start.hasFailed()) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/RuntimeRegistration.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/RuntimeRegistration.java
index 0c4b806..0eba381 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/RuntimeRegistration.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/RuntimeRegistration.java
@@ -49,7 +49,7 @@
             subscriber.beforeExecute();
         }
         ActiveEvent event = new ActiveEvent(jobId, Kind.PARTITION_EVENT, entityId, new ActivePartitionMessage(
-                new ActiveRuntimeId(entityId, nc.getId(), partition), jobId, Event.RUNTIME_REGISTERED, null));
+                new ActiveRuntimeId(entityId, nc.getId(), partition), jobId, Event.RUNTIME_REGISTERED, null, ""));
         nc.getClusterController().activeEvent(event);
     }
 
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestNodeControllerActor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestNodeControllerActor.java
index f849f08..fcd4d85 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestNodeControllerActor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestNodeControllerActor.java
@@ -92,7 +92,7 @@
                     subscriber.beforeExecute();
                 }
                 ActiveEvent event = new ActiveEvent(jobId, Kind.PARTITION_EVENT, entityId, new ActivePartitionMessage(
-                        new ActiveRuntimeId(entityId, id, partition), jobId, Event.RUNTIME_DEREGISTERED, null));
+                        new ActiveRuntimeId(entityId, id, partition), jobId, Event.RUNTIME_DEREGISTERED, null, ""));
                 clusterController.activeEvent(event);
             }
         };
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java
index 2d64f79..fa2c43e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java
@@ -263,7 +263,7 @@
         Action action = new Action() {
             @Override
             protected void doExecute(MetadataProvider mdProvider) throws Exception {
-                handler.suspend(mdProvider);
+                handler.suspend(mdProvider, "");
             }
         };
         add(action);
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/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 93031c5..6eaa09d 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/ProfiledExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ProfiledExecutionTest.java
index 7c945fc..413f002 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ProfiledExecutionTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ProfiledExecutionTest.java
@@ -30,9 +30,6 @@
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-/**
- * Runs the cluster state runtime tests with the storage parallelism.
- */
 @RunWith(Parameterized.class)
 public class ProfiledExecutionTest {
     protected static final String TEST_CONFIG_FILE_NAME = "src/test/resources/cc-single.conf";
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..56169e7
--- /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.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/java/org/apache/asterix/test/storage/DeallocatableTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DeallocatableTest.java
index cfd251b..368b17b 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DeallocatableTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DeallocatableTest.java
@@ -22,9 +22,9 @@
 import java.util.concurrent.TimeUnit;
 
 import org.apache.asterix.app.bootstrap.TestNodeController;
-import org.apache.asterix.replication.management.NetworkingUtil;
 import org.apache.asterix.test.common.TestHelper;
 import org.apache.asterix.test.runtime.LangExecutionUtil;
+import org.apache.hyracks.api.comm.NetworkAddress;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.ConnectorDescriptorId;
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
@@ -63,8 +63,8 @@
             final IHyracksTaskContext ctx = nc.createTestContext(jobId, 0, true);
             final ConnectorDescriptorId codId = new ConnectorDescriptorId(1);
             final PartitionId pid = new PartitionId(ctx.getJobletContext().getJobId(), codId, 1, 1);
-            final ChannelControlBlock ccb = ncs.getNetworkManager()
-                    .connect(NetworkingUtil.getSocketAddress(ncs.getNetworkManager().getLocalNetworkAddress()));
+            NetworkAddress netAddr = ncs.getNetworkManager().getLocalNetworkAddress();
+            final ChannelControlBlock ccb = ncs.getNetworkManager().connect(netAddr.toResolvedInetSocketAddress());
             final NetworkOutputChannel networkOutputChannel = new NetworkOutputChannel(ccb, 0);
             final MaterializingPipelinedPartition mpp =
                     new MaterializingPipelinedPartition(ctx, ncs.getPartitionManager(), pid, taId, ncs.getExecutor());
diff --git a/asterixdb/asterix-app/src/test/resources/cc-analyze.conf b/asterixdb/asterix-app/src/test/resources/cc-analyze.conf
index d8b3512..16fa0ae 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-analyze.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-analyze.conf
@@ -1,5 +1,3 @@
-<<<<<<< HEAD   (5f076a [ASTERIXDB-3221][REPL] Use IO scheduler for replication ops)
-=======
 ; Licensed to the Apache Software Foundation (ASF) under one
 ; or more contributor license agreements.  See the NOTICE file
 ; distributed with this work for additional information
@@ -56,4 +54,3 @@
 log.level = INFO
 compiler.groupmemory=64MB
 storage.buffercache.pagesize=32KB
->>>>>>> CHANGE (337517 [NO ISSUE][OTH] Move page size storage config to common conf)
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..7d3aa52
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf
@@ -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.
+
+[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.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
+storage.buffercache.pagesize=32KB
diff --git a/asterixdb/asterix-app/src/test/resources/cc-cloud-storage.conf b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage.conf
index 18627b4..6373eae 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-cloud-storage.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-cloud-storage.conf
@@ -1,5 +1,3 @@
-<<<<<<< HEAD   (5f076a [ASTERIXDB-3221][REPL] Use IO scheduler for replication ops)
-=======
 ; Licensed to the Apache Software Foundation (ASF) under one
 ; or more contributor license agreements.  See the NOTICE file
 ; distributed with this work for additional information
@@ -64,4 +62,3 @@
 messaging.frame.count=512
 cloud.deployment=true
 storage.buffercache.pagesize=32KB
->>>>>>> CHANGE (337517 [NO ISSUE][OTH] Move page size storage config to common conf)
diff --git a/asterixdb/asterix-app/src/test/resources/cc-compression.conf b/asterixdb/asterix-app/src/test/resources/cc-compression.conf
index 373d9bb..bc4ea4a 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-compression.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-compression.conf
@@ -35,7 +35,6 @@
 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
 
@@ -58,3 +57,4 @@
 messaging.frame.size=4096
 messaging.frame.count=512
 storage.compression.block=snappy
+storage.buffercache.pagesize=32KB
diff --git a/asterixdb/asterix-app/src/test/resources/cc-single-lookup.conf b/asterixdb/asterix-app/src/test/resources/cc-single-lookup.conf
index 1b006b3..d59d4d5 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-single-lookup.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-single-lookup.conf
@@ -1,5 +1,3 @@
-<<<<<<< HEAD   (5f076a [ASTERIXDB-3221][REPL] Use IO scheduler for replication ops)
-=======
 ; Licensed to the Apache Software Foundation (ASF) under one
 ; or more contributor license agreements.  See the NOTICE file
 ; distributed with this work for additional information
@@ -64,4 +62,3 @@
 messaging.frame.size=4096
 messaging.frame.count=512
 storage.buffercache.pagesize=32KB
->>>>>>> CHANGE (337517 [NO ISSUE][OTH] Move page size storage config to common conf)
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.1.sqlpp
similarity index 62%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.1.sqlpp
index 6afbccb..76f8416 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.1.sqlpp
@@ -16,17 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+/**
+ * Index on top-level field, single OBJECT_ADD function application.
+ * Index "usersNameIdx" should be used.
+ */
 
-    void pause();
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
 
-    void resume();
-}
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM   Test.Users U
+LET    augmentedUser = OBJECT_ADD(U, "favoriteColor", "Green")
+WHERE  augmentedUser.name = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.10.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.10.sqlpp
new file mode 100644
index 0000000..7e018b9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.10.sqlpp
@@ -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.
+ */
+
+/**
+ * Index on top-level field, three OBJECT_ADD function applications.
+ * Primary index should used w/ INLJ.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+
+FROM   Test.Users U1, Test.Users U2
+LET    augmentedUser1 = OBJECT_ADD(U1, "favoriteColor", "Green"),
+       augmentedUser2 = OBJECT_ADD(U2, "favoriteFood", "Pizza"),
+       augmentedUser3 = OBJECT_ADD(augmentedUser2, "favoriteColor", "Red"),
+       augmentedUser4 = OBJECT_ADD(augmentedUser3, "favoriteDrink", "Wine")
+WHERE  augmentedUser1.name = "John" AND
+       augmentedUser2.name = "Sally" AND
+       TO_BIGINT(U1.bestFriend) /* +indexnl */ = augmentedUser4.user_id
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.11.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.11.sqlpp
new file mode 100644
index 0000000..782060f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.11.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Two indexes on nested fields, one OBJECT_ADD function application each.
+ * Index should be used in both cases.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint, name: { first: string } };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX firstUsersNameIdx ON Users ( name.first );
+CREATE INDEX lastUsersNameIdx ON Users ( name.last: string );
+
+FROM   Test.Users U1
+LET    augmentedUser1 = OBJECT_ADD(U1, "favoriteColor", "Green")
+WHERE  augmentedUser1.name.first = "Glenn"
+SELECT augmentedUser1.*
+
+UNION ALL
+
+FROM   Test.Users U2
+LET    augmentedUser2 = OBJECT_ADD(U2, "favoriteFood", "Pizza")
+WHERE  augmentedUser2.name.last = "John"
+SELECT augmentedUser2.*;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.2.sqlpp
new file mode 100644
index 0000000..4583e1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.2.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Index on top-level field, two OBJECT_ADD function applications.
+ * Index "usersNameIdx" should be used.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM   Test.Users U
+LET    augmentedUser1 = OBJECT_ADD(U, "favoriteColor", "Green"),
+       augmentedUser2 = OBJECT_ADD(augmentedUser1, "favoriteCity", "Irvine")
+WHERE  augmentedUser2.name = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.3.sqlpp
similarity index 62%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.3.sqlpp
index 6afbccb..0d81c0f 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.3.sqlpp
@@ -16,17 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+/**
+ * Index on nested field, one OBJECT_ADD function application.
+ * Index "usersNameIdx" should be used.
+ */
 
-    void pause();
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
 
-    void resume();
-}
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name.first : string );
+
+FROM   Test.Users U
+LET    augmentedUser = OBJECT_ADD(U, "favoriteColor", "Green")
+WHERE  augmentedUser.name.first = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.4.sqlpp
similarity index 63%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.4.sqlpp
index 6afbccb..95a60e6 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.4.sqlpp
@@ -16,17 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+/**
+ * Index on top-level field, one OBJECT_REMOVE function application.
+ * Index "usersNameIdx" should be used.
+ */
 
-    void pause();
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
 
-    void resume();
-}
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM   Test.Users U
+LET    augmentedUser = OBJECT_REMOVE(U, "favoriteColor")
+WHERE  augmentedUser.name = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.5.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.5.sqlpp
index 6afbccb..41af922 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.5.sqlpp
@@ -16,17 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+/**
+ * Index on top-level field, one OBJECT_PUT function application.
+ * Index should NOT used.
+ */
 
-    void pause();
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
 
-    void resume();
-}
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM   Test.Users U
+LET    augmentedUser = OBJECT_PUT(U, "name", "John")
+WHERE  augmentedUser.name = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.6.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.6.sqlpp
new file mode 100644
index 0000000..8b573b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.6.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Index on top-level field, one OBJECT_ADD function application.
+ * Primary index should used w/ INLJ.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+
+FROM   Test.Users U1, Test.Users U2
+LET    augmentedUser1 = OBJECT_ADD(U1, "favoriteColor", "Green"),
+       augmentedUser2 = OBJECT_ADD(U2, "favoriteFood", "Pizza")
+WHERE  augmentedUser1.name = "John" AND
+       augmentedUser2.name = "Sally" AND
+       TO_BIGINT(U1.bestFriend) /* +indexnl */ = augmentedUser2.user_id
+SELECT *;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.7.sqlpp
similarity index 62%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.7.sqlpp
index 6afbccb..cb48c1c 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.7.sqlpp
@@ -16,17 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+/**
+ * Index on top-level field, OBJECT_ADD followed by OBJECT_REMOVE.
+ * Index should NOT be used.
+ */
 
-    void pause();
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
 
-    void resume();
-}
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM   Test.Users U
+LET    augmentedUser = OBJECT_ADD(OBJECT_REMOVE(U, "name"), "name", "Glenn")
+WHERE  augmentedUser.firstName = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.8.sqlpp
similarity index 60%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.8.sqlpp
index 6afbccb..8d780bf 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.8.sqlpp
@@ -16,17 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+/**
+ * Index on top-level field, single OBJECT_CONCAT function application with multiple records.
+ * Index "usersNameIdx" should be used.
+ */
 
-    void pause();
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
 
-    void resume();
-}
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM   Test.Users U
+LET    augmentedUser = OBJECT_CONCAT({"favoriteColor": "Green"}, U, {"birthdate": "10/09/1996"})
+WHERE  augmentedUser.name = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.9.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.9.sqlpp
new file mode 100644
index 0000000..ad827e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.9.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.
+ */
+
+/**
+ * Index on top-level field, three OBJECT_ADD function applications.
+ * Index should be used twice (and not used for E1).
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM   Test.Users U1, Test.Users U2, [{"name": "Glenn"}] E1
+LET    augmentedUser1 = OBJECT_ADD(U1, "favoriteColor", "Green"),
+       augmentedUser2 = OBJECT_ADD(U2, "favoriteFood", "Pizza"),
+       augmentedUser3 = OBJECT_ADD(E1, "favoriteColor", "Blue")
+WHERE  augmentedUser1.name = "John" AND
+       augmentedUser2.name = "Sally" AND
+       augmentedUser3.name = "Glenn" AND
+       augmentedUser1.bestFriend = augmentedUser2.user_id
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue3316.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue3316.sqlpp
new file mode 100644
index 0000000..cc9bb60
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue3316.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : This test case is to verify the fix for issue 3316
+ * Expected Res : SUCCESS
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+create type dt1 as {id:int};
+
+create dataset collection0(dt1) primary key id;
+create dataset collection1(dt1) primary key id;
+
+SELECT DISTINCT T1.*,
+        (SELECT H.u
+                   FROM collection1 H
+                  WHERE H.y_id = T1.x_id
+                    AND H.a IN [12, 66, 67, 13, 26]
+                    AND H.to_u = 'aaaaa'
+                    AND H.b in
+                        (SELECT value MAX(L.b)
+                           FROM collection1 L
+                          WHERE L.y_id = T1.x_id
+                            AND L.a IN [12, 66, 67, 13, 26]
+                            AND L.to_u = 'aaaaa')
+                            ) AS sub_query1,
+        (SELECT H.u
+                   FROM collection1 H
+                  WHERE  H.y_id = T1.x_id
+                    AND H.a IN [12, 66, 67, 13]
+                    AND H.posi IN ['a','b']
+                    AND H.b in
+                        (SELECT value MAX(L.b)
+                           FROM collection1 L
+                          WHERE L.y_id = T1.x_id
+                            AND L.posi IN ['a','b'])
+                                  ) AS sub_query2
+FROM (SELECT T0.x_id
+        FROM collection0 T0
+) T1;
+
+
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 efd14f7..b596e4b 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|
@@ -35,27 +35,27 @@
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$252]  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$267]  |PARTITIONED|
                                           {
                                             -- AGGREGATE  |LOCAL|
                                               -- STREAM_SELECT  |LOCAL|
                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                                           }
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$252(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$252]  |PARTITIONED|
+                                      -- STABLE_SORT [$$267(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$267]  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- HYBRID_HASH_JOIN [$$251][$$222]  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$266][$$237]  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                   -- RUNNING_AGGREGATE  |PARTITIONED|
                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                       -- UNNEST  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- PRE_CLUSTERED_GROUP_BY[$$313]  |PARTITIONED|
+                                                          -- PRE_CLUSTERED_GROUP_BY[$$322]  |PARTITIONED|
                                                                   {
                                                                     -- AGGREGATE  |LOCAL|
-                                                                      -- MICRO_PRE_CLUSTERED_GROUP_BY[$$315, $$316]  |LOCAL|
+                                                                      -- MICRO_PRE_CLUSTERED_GROUP_BY[$$324, $$325]  |LOCAL|
                                                                               {
                                                                                 -- AGGREGATE  |LOCAL|
                                                                                   -- STREAM_SELECT  |LOCAL|
@@ -65,8 +65,8 @@
                                                                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                                   }
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- STABLE_SORT [$$313(ASC), $$315(ASC), $$316(ASC)]  |PARTITIONED|
-                                                                -- HASH_PARTITION_EXCHANGE [$$313]  |PARTITIONED|
+                                                              -- STABLE_SORT [$$322(ASC), $$324(ASC), $$325(ASC)]  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$322]  |PARTITIONED|
                                                                   -- UNION_ALL  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                       -- STREAM_PROJECT  |PARTITIONED|
@@ -102,7 +102,7 @@
                                                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                         -- BTREE_SEARCH (channels.Reports.Reports)  |PARTITIONED|
                                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                            -- STABLE_SORT [$$259(ASC)]  |PARTITIONED|
+                                                                                                                            -- STABLE_SORT [$$274(ASC)]  |PARTITIONED|
                                                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -147,7 +147,7 @@
                                                                                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                   -- BTREE_SEARCH (channels.Reports.Reports)  |PARTITIONED|
                                                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                      -- STABLE_SORT [$$259(ASC)]  |PARTITIONED|
+                                                                                                                      -- STABLE_SORT [$$274(ASC)]  |PARTITIONED|
                                                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                           -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -161,12 +161,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/index-through-object/index-through-object.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.1.plan
new file mode 100644
index 0000000..ff509f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.1.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.10.plan
new file mode 100644
index 0000000..f5ab2c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.10.plan
@@ -0,0 +1,20 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$80]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.11.plan
new file mode 100644
index 0000000..6971ab4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.11.plan
@@ -0,0 +1,39 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- UNION_ALL  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$78(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (Test.Users.firstUsersNameIdx)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$82(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (Test.Users.lastUsersNameIdx)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.2.plan
new file mode 100644
index 0000000..90816ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.2.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.3.plan
new file mode 100644
index 0000000..6930560
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.3.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.4.plan
new file mode 100644
index 0000000..ff509f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.4.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.5.plan
new file mode 100644
index 0000000..f0f6a03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.5.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.6.plan
new file mode 100644
index 0000000..b4aee6b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.6.plan
@@ -0,0 +1,20 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.7.plan
new file mode 100644
index 0000000..f0f6a03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.7.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.8.plan
new file mode 100644
index 0000000..6930560
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.8.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.9.plan
new file mode 100644
index 0000000..1d2e55b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.9.plan
@@ -0,0 +1,47 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |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 [$$82][$$83]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$82]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$106(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$110(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- BROADCAST_EXCHANGE  |PARTITIONED|
+              -- STREAM_SELECT  |UNPARTITIONED|
+                -- ASSIGN  |UNPARTITIONED|
+                  -- UNNEST  |UNPARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
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_issue3316.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue3316.plan
new file mode 100644
index 0000000..5d1f352
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue3316.plan
@@ -0,0 +1,1000 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$194(ASC)]  |PARTITIONED|
+          -- HASH_PARTITION_EXCHANGE [$$194]  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$229]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$229(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$229][$$390]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- PRE_CLUSTERED_GROUP_BY[$$195]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- STREAM_SELECT  |LOCAL|
+                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$195][$$242]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$242]  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- HYBRID_HASH_JOIN [$$128][$$201]  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- PRE_CLUSTERED_GROUP_BY[$$227, $$196]  |PARTITIONED|
+                                                                            {
+                                                                              -- AGGREGATE  |LOCAL|
+                                                                                -- STREAM_SELECT  |LOCAL|
+                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                            }
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- STABLE_SORT [$$227(ASC), $$196(ASC)]  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- HYBRID_HASH_JOIN [$$227, $$196][$$256, $$257]  |PARTITIONED|
+                                                                                  -- HASH_PARTITION_EXCHANGE [$$227, $$196]  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- HYBRID_HASH_JOIN [$$225][$$209]  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                    -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                          -- HASH_PARTITION_EXCHANGE [$$209]  |PARTITIONED|
+                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                            -- UNNEST  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- PRE_CLUSTERED_GROUP_BY[$$240, $$237]  |PARTITIONED|
+                                                                                                        {
+                                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                                              -- STREAM_SELECT  |LOCAL|
+                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                        }
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- STABLE_SORT [$$240(ASC), $$237(ASC)]  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- HYBRID_HASH_JOIN [$$240, $$237][$$254, $$251]  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$427, $$428]  |PARTITIONED|
+                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- HYBRID_HASH_JOIN [$$430][$$429]  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$429]  |PARTITIONED|
+                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                              -- HASH_PARTITION_EXCHANGE [$$254, $$251]  |PARTITIONED|
+                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- HYBRID_HASH_JOIN [$$137][$$200]  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- HYBRID_HASH_JOIN [$$245][$$212]  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |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|
+                                                                                                                                                -- HYBRID_HASH_JOIN [$$447][$$450]  |PARTITIONED|
+                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                              -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$450]  |PARTITIONED|
+                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
+                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- UNNEST  |UNPARTITIONED|
+                                                                                                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            -- UNNEST  |UNPARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$390]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$164][$$204]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$167][$$203]  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- PRE_CLUSTERED_GROUP_BY[$$258, $$198]  |PARTITIONED|
+                                                                        {
+                                                                          -- AGGREGATE  |LOCAL|
+                                                                            -- STREAM_SELECT  |LOCAL|
+                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                        }
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- STABLE_SORT [$$258(ASC), $$198(ASC)]  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- HYBRID_HASH_JOIN [$$258, $$198][$$461, $$462]  |PARTITIONED|
+                                                                              -- HASH_PARTITION_EXCHANGE [$$258, $$198]  |PARTITIONED|
+                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- HYBRID_HASH_JOIN [$$262][$$216]  |PARTITIONED|
+                                                                                      -- HASH_PARTITION_EXCHANGE [$$262]  |PARTITIONED|
+                                                                                        -- PRE_CLUSTERED_GROUP_BY[$$259]  |PARTITIONED|
+                                                                                                {
+                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                }
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- HYBRID_HASH_JOIN [$$259][$$263]  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                        -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                  -- HASH_PARTITION_EXCHANGE [$$263]  |PARTITIONED|
+                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- HYBRID_HASH_JOIN [$$268][$$267]  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$270, $$272]  |PARTITIONED|
+                                                                                                                              {
+                                                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                                                  -- STREAM_SELECT  |LOCAL|
+                                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                              }
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- STABLE_SORT [$$270(ASC), $$272(ASC)]  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- HYBRID_HASH_JOIN [$$270, $$272][$$280, $$281]  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$411, $$413]  |PARTITIONED|
+                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                    -- HYBRID_HASH_JOIN [$$416][$$417]  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                              -- UNNEST  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- PRE_CLUSTERED_GROUP_BY[$$286, $$287]  |PARTITIONED|
+                                                                                                                                                          {
+                                                                                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                                                                                              -- AGGREGATE  |LOCAL|
+                                                                                                                                                                -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                                          }
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- STABLE_SORT [$$286(ASC), $$287(ASC)]  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                              -- HYBRID_HASH_JOIN [$$286, $$287][$$296, $$297]  |PARTITIONED|
+                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$427, $$428]  |PARTITIONED|
+                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- HYBRID_HASH_JOIN [$$430][$$429]  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                              -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$429]  |PARTITIONED|
+                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$296, $$297]  |PARTITIONED|
+                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- HYBRID_HASH_JOIN [$$304][$$303]  |PARTITIONED|
+                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- HYBRID_HASH_JOIN [$$306][$$305]  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |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|
+                                                                                                                                                                                                  -- HYBRID_HASH_JOIN [$$447][$$450]  |PARTITIONED|
+                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$450]  |PARTITIONED|
+                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |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 [$$446]  |PARTITIONED|
+                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                            -- UNNEST  |UNPARTITIONED|
+                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                              -- UNNEST  |UNPARTITIONED|
+                                                                                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                      -- HASH_PARTITION_EXCHANGE [$$216]  |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 (test.collection1)  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- UNNEST  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- PRE_CLUSTERED_GROUP_BY[$$329, $$328]  |PARTITIONED|
+                                                                                                    {
+                                                                                                      -- AGGREGATE  |LOCAL|
+                                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                                          -- STREAM_SELECT  |LOCAL|
+                                                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                    }
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- STABLE_SORT [$$329(ASC), $$328(ASC)]  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- HYBRID_HASH_JOIN [$$329, $$328][$$400, $$399]  |PARTITIONED|
+                                                                                                          -- HASH_PARTITION_EXCHANGE [$$329, $$328]  |PARTITIONED|
+                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- HYBRID_HASH_JOIN [$$322][$$321]  |PARTITIONED|
+                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$322]  |PARTITIONED|
+                                                                                                                    -- PRE_CLUSTERED_GROUP_BY[$$330]  |PARTITIONED|
+                                                                                                                            {
+                                                                                                                              -- AGGREGATE  |LOCAL|
+                                                                                                                                -- STREAM_SELECT  |LOCAL|
+                                                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                            }
+                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- HYBRID_HASH_JOIN [$$330][$$333]  |PARTITIONED|
+                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                    -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$333]  |PARTITIONED|
+                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- HYBRID_HASH_JOIN [$$338][$$337]  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- PRE_CLUSTERED_GROUP_BY[$$340, $$342]  |PARTITIONED|
+                                                                                                                                                          {
+                                                                                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                                                                                              -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                                          }
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- STABLE_SORT [$$340(ASC), $$342(ASC)]  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                              -- HYBRID_HASH_JOIN [$$340, $$342][$$350, $$351]  |PARTITIONED|
+                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$411, $$413]  |PARTITIONED|
+                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- HYBRID_HASH_JOIN [$$416][$$417]  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                          -- UNNEST  |PARTITIONED|
+                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                              -- PRE_CLUSTERED_GROUP_BY[$$356, $$357]  |PARTITIONED|
+                                                                                                                                                                                      {
+                                                                                                                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                                                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                                                                                                                            -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                                                                      }
+                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                  -- STABLE_SORT [$$356(ASC), $$357(ASC)]  |PARTITIONED|
+                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                          -- HYBRID_HASH_JOIN [$$356, $$357][$$366, $$367]  |PARTITIONED|
+                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$427, $$428]  |PARTITIONED|
+                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$430][$$429]  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$429]  |PARTITIONED|
+                                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                      -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$366, $$367]  |PARTITIONED|
+                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                    -- HYBRID_HASH_JOIN [$$374][$$373]  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$376][$$375]  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |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|
+                                                                                                                                                                                                                              -- HYBRID_HASH_JOIN [$$447][$$450]  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$450]  |PARTITIONED|
+                                                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                        -- DATASOURCE_SCAN (test.collection1)  |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 [$$446]  |PARTITIONED|
+                                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- UNNEST  |UNPARTITIONED|
+                                                                                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- UNNEST  |UNPARTITIONED|
+                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$321]  |PARTITIONED|
+                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                          -- HASH_PARTITION_EXCHANGE [$$400, $$399]  |PARTITIONED|
+                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- HYBRID_HASH_JOIN [$$174][$$202]  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- HYBRID_HASH_JOIN [$$393][$$218]  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- HYBRID_HASH_JOIN [$$393][$$392]  |PARTITIONED|
+                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$393]  |PARTITIONED|
+                                                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$401]  |PARTITIONED|
+                                                                                                                                              {
+                                                                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                                                                  -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                              }
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- HYBRID_HASH_JOIN [$$401][$$404]  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$404]  |PARTITIONED|
+                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- HYBRID_HASH_JOIN [$$409][$$408]  |PARTITIONED|
+                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                    -- PRE_CLUSTERED_GROUP_BY[$$411, $$413]  |PARTITIONED|
+                                                                                                                                                                            {
+                                                                                                                                                                              -- AGGREGATE  |LOCAL|
+                                                                                                                                                                                -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                                                            }
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- STABLE_SORT [$$411(ASC), $$413(ASC)]  |PARTITIONED|
+                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- HYBRID_HASH_JOIN [$$411, $$413][$$421, $$422]  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$411, $$413]  |PARTITIONED|
+                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$416][$$417]  |PARTITIONED|
+                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                            -- UNNEST  |PARTITIONED|
+                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                -- PRE_CLUSTERED_GROUP_BY[$$427, $$428]  |PARTITIONED|
+                                                                                                                                                                                                        {
+                                                                                                                                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                                                                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                                                                                                                                              -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                                                                                        }
+                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                    -- STABLE_SORT [$$427(ASC), $$428(ASC)]  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$427, $$428][$$437, $$438]  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$427, $$428]  |PARTITIONED|
+                                                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                        -- HYBRID_HASH_JOIN [$$430][$$429]  |PARTITIONED|
+                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                      -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$429]  |PARTITIONED|
+                                                                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$437, $$438]  |PARTITIONED|
+                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                      -- HYBRID_HASH_JOIN [$$445][$$444]  |PARTITIONED|
+                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                              -- HYBRID_HASH_JOIN [$$447][$$446]  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                          -- HYBRID_HASH_JOIN [$$447][$$450]  |PARTITIONED|
+                                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                                        -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$450]  |PARTITIONED|
+                                                                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$446]  |PARTITIONED|
+                                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                          -- UNNEST  |UNPARTITIONED|
+                                                                                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- UNNEST  |UNPARTITIONED|
+                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$392]  |PARTITIONED|
+                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$218]  |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|
+                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- UNNEST  |UNPARTITIONED|
+                                                                                                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- UNNEST  |UNPARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
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/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/index-through-object/index-through-object.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.1.plan
new file mode 100644
index 0000000..f0f6a03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.1.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.2.plan
new file mode 100644
index 0000000..f0f6a03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.2.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.3.plan
new file mode 100644
index 0000000..f0f6a03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.3.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.4.plan
new file mode 100644
index 0000000..f0f6a03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.4.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.8.plan
new file mode 100644
index 0000000..f0f6a03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.8.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.9.plan
new file mode 100644
index 0000000..75a50b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/index-through-object/index-through-object.9.plan
@@ -0,0 +1,41 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |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 [$$88][$$89]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$88]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |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.Users)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$89]  |PARTITIONED|
+                      -- STREAM_SELECT  |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.Users)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- BROADCAST_EXCHANGE  |PARTITIONED|
+              -- STREAM_SELECT  |UNPARTITIONED|
+                -- ASSIGN  |UNPARTITIONED|
+                  -- 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
index 91a6aca..5f3c681 100644
--- 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
@@ -6,26 +6,24 @@
           -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
             -- AGGREGATE  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- NESTED_LOOP  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$76][$$78]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                    -- NESTED_LOOP  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- HYBRID_HASH_JOIN [$$76][$$78]  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
+                            -- DATASOURCE_SCAN (tpch.Supplier)  |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|
+                                -- 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|
-                                -- DATASOURCE_SCAN (tpch.Partsupp)  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (tpch.Part)  |PARTITIONED|
+                        -- DATASOURCE_SCAN (tpch.Partsupp)  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01.plan
index 2326c55..e4167f0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01.plan
@@ -15,7 +15,7 @@
   -- 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|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01_ps.plan
index 4bdc66b..8e1370f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/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|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02.plan
index eb1d57c..c87d9ce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02.plan
@@ -15,7 +15,7 @@
   -- 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|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02_ps.plan
index dce703e..6db1de8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive-open_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/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|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01.plan
index 2326c55..e4167f0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01.plan
@@ -15,7 +15,7 @@
   -- 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|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01_ps.plan
index 4bdc66b..8e1370f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/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|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02.plan
index eb1d57c..c87d9ce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02.plan
@@ -15,7 +15,7 @@
   -- 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|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02_ps.plan
index dce703e..6db1de8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/orders-index-search-conjunctive_02_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/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|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue3316.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue3316.plan
new file mode 100644
index 0000000..5d1f352
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/query_issue3316.plan
@@ -0,0 +1,1000 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$194(ASC)]  |PARTITIONED|
+          -- HASH_PARTITION_EXCHANGE [$$194]  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$229]  |PARTITIONED|
+                            {
+                              -- AGGREGATE  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                            }
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$229(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$229][$$390]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- PRE_CLUSTERED_GROUP_BY[$$195]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- STREAM_SELECT  |LOCAL|
+                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$195][$$242]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$242]  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- HYBRID_HASH_JOIN [$$128][$$201]  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- PRE_CLUSTERED_GROUP_BY[$$227, $$196]  |PARTITIONED|
+                                                                            {
+                                                                              -- AGGREGATE  |LOCAL|
+                                                                                -- STREAM_SELECT  |LOCAL|
+                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                            }
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- STABLE_SORT [$$227(ASC), $$196(ASC)]  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- HYBRID_HASH_JOIN [$$227, $$196][$$256, $$257]  |PARTITIONED|
+                                                                                  -- HASH_PARTITION_EXCHANGE [$$227, $$196]  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- HYBRID_HASH_JOIN [$$225][$$209]  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                    -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                          -- HASH_PARTITION_EXCHANGE [$$209]  |PARTITIONED|
+                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                            -- UNNEST  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- PRE_CLUSTERED_GROUP_BY[$$240, $$237]  |PARTITIONED|
+                                                                                                        {
+                                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                                              -- STREAM_SELECT  |LOCAL|
+                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                        }
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- STABLE_SORT [$$240(ASC), $$237(ASC)]  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- HYBRID_HASH_JOIN [$$240, $$237][$$254, $$251]  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$427, $$428]  |PARTITIONED|
+                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- HYBRID_HASH_JOIN [$$430][$$429]  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$429]  |PARTITIONED|
+                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                              -- HASH_PARTITION_EXCHANGE [$$254, $$251]  |PARTITIONED|
+                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- HYBRID_HASH_JOIN [$$137][$$200]  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- HYBRID_HASH_JOIN [$$245][$$212]  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |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|
+                                                                                                                                                -- HYBRID_HASH_JOIN [$$447][$$450]  |PARTITIONED|
+                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                              -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$450]  |PARTITIONED|
+                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$212]  |PARTITIONED|
+                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- UNNEST  |UNPARTITIONED|
+                                                                                                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            -- UNNEST  |UNPARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$390]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$164][$$204]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$167][$$203]  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- PRE_CLUSTERED_GROUP_BY[$$258, $$198]  |PARTITIONED|
+                                                                        {
+                                                                          -- AGGREGATE  |LOCAL|
+                                                                            -- STREAM_SELECT  |LOCAL|
+                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                        }
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- STABLE_SORT [$$258(ASC), $$198(ASC)]  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- HYBRID_HASH_JOIN [$$258, $$198][$$461, $$462]  |PARTITIONED|
+                                                                              -- HASH_PARTITION_EXCHANGE [$$258, $$198]  |PARTITIONED|
+                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- HYBRID_HASH_JOIN [$$262][$$216]  |PARTITIONED|
+                                                                                      -- HASH_PARTITION_EXCHANGE [$$262]  |PARTITIONED|
+                                                                                        -- PRE_CLUSTERED_GROUP_BY[$$259]  |PARTITIONED|
+                                                                                                {
+                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                }
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- HYBRID_HASH_JOIN [$$259][$$263]  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                        -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                  -- HASH_PARTITION_EXCHANGE [$$263]  |PARTITIONED|
+                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- HYBRID_HASH_JOIN [$$268][$$267]  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$270, $$272]  |PARTITIONED|
+                                                                                                                              {
+                                                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                                                  -- STREAM_SELECT  |LOCAL|
+                                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                              }
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- STABLE_SORT [$$270(ASC), $$272(ASC)]  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- HYBRID_HASH_JOIN [$$270, $$272][$$280, $$281]  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$411, $$413]  |PARTITIONED|
+                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                    -- HYBRID_HASH_JOIN [$$416][$$417]  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                              -- UNNEST  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- PRE_CLUSTERED_GROUP_BY[$$286, $$287]  |PARTITIONED|
+                                                                                                                                                          {
+                                                                                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                                                                                              -- AGGREGATE  |LOCAL|
+                                                                                                                                                                -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                                          }
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- STABLE_SORT [$$286(ASC), $$287(ASC)]  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                              -- HYBRID_HASH_JOIN [$$286, $$287][$$296, $$297]  |PARTITIONED|
+                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$427, $$428]  |PARTITIONED|
+                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- HYBRID_HASH_JOIN [$$430][$$429]  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                              -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$429]  |PARTITIONED|
+                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$296, $$297]  |PARTITIONED|
+                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- HYBRID_HASH_JOIN [$$304][$$303]  |PARTITIONED|
+                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- HYBRID_HASH_JOIN [$$306][$$305]  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |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|
+                                                                                                                                                                                                  -- HYBRID_HASH_JOIN [$$447][$$450]  |PARTITIONED|
+                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$450]  |PARTITIONED|
+                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |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 [$$446]  |PARTITIONED|
+                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                            -- UNNEST  |UNPARTITIONED|
+                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                              -- UNNEST  |UNPARTITIONED|
+                                                                                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                      -- HASH_PARTITION_EXCHANGE [$$216]  |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 (test.collection1)  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- UNNEST  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- PRE_CLUSTERED_GROUP_BY[$$329, $$328]  |PARTITIONED|
+                                                                                                    {
+                                                                                                      -- AGGREGATE  |LOCAL|
+                                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                                          -- STREAM_SELECT  |LOCAL|
+                                                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                    }
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- STABLE_SORT [$$329(ASC), $$328(ASC)]  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- HYBRID_HASH_JOIN [$$329, $$328][$$400, $$399]  |PARTITIONED|
+                                                                                                          -- HASH_PARTITION_EXCHANGE [$$329, $$328]  |PARTITIONED|
+                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- HYBRID_HASH_JOIN [$$322][$$321]  |PARTITIONED|
+                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$322]  |PARTITIONED|
+                                                                                                                    -- PRE_CLUSTERED_GROUP_BY[$$330]  |PARTITIONED|
+                                                                                                                            {
+                                                                                                                              -- AGGREGATE  |LOCAL|
+                                                                                                                                -- STREAM_SELECT  |LOCAL|
+                                                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                            }
+                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                            -- HYBRID_HASH_JOIN [$$330][$$333]  |PARTITIONED|
+                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                    -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$333]  |PARTITIONED|
+                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- HYBRID_HASH_JOIN [$$338][$$337]  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- PRE_CLUSTERED_GROUP_BY[$$340, $$342]  |PARTITIONED|
+                                                                                                                                                          {
+                                                                                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                                                                                              -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                                          }
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- STABLE_SORT [$$340(ASC), $$342(ASC)]  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                              -- HYBRID_HASH_JOIN [$$340, $$342][$$350, $$351]  |PARTITIONED|
+                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$411, $$413]  |PARTITIONED|
+                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- HYBRID_HASH_JOIN [$$416][$$417]  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                          -- UNNEST  |PARTITIONED|
+                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                              -- PRE_CLUSTERED_GROUP_BY[$$356, $$357]  |PARTITIONED|
+                                                                                                                                                                                      {
+                                                                                                                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                                                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                                                                                                                            -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                                                                      }
+                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                  -- STABLE_SORT [$$356(ASC), $$357(ASC)]  |PARTITIONED|
+                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                          -- HYBRID_HASH_JOIN [$$356, $$357][$$366, $$367]  |PARTITIONED|
+                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$427, $$428]  |PARTITIONED|
+                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$430][$$429]  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                        -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                          -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$429]  |PARTITIONED|
+                                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                      -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$366, $$367]  |PARTITIONED|
+                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                    -- HYBRID_HASH_JOIN [$$374][$$373]  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$376][$$375]  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |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|
+                                                                                                                                                                                                                              -- HYBRID_HASH_JOIN [$$447][$$450]  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$450]  |PARTITIONED|
+                                                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                        -- DATASOURCE_SCAN (test.collection1)  |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 [$$446]  |PARTITIONED|
+                                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- UNNEST  |UNPARTITIONED|
+                                                                                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- UNNEST  |UNPARTITIONED|
+                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$321]  |PARTITIONED|
+                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                          -- HASH_PARTITION_EXCHANGE [$$400, $$399]  |PARTITIONED|
+                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- HYBRID_HASH_JOIN [$$174][$$202]  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- HYBRID_HASH_JOIN [$$393][$$218]  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- HYBRID_HASH_JOIN [$$393][$$392]  |PARTITIONED|
+                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$393]  |PARTITIONED|
+                                                                                                                                      -- PRE_CLUSTERED_GROUP_BY[$$401]  |PARTITIONED|
+                                                                                                                                              {
+                                                                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                                                                  -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                              }
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                              -- HYBRID_HASH_JOIN [$$401][$$404]  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                -- HASH_PARTITION_EXCHANGE [$$404]  |PARTITIONED|
+                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- HYBRID_HASH_JOIN [$$409][$$408]  |PARTITIONED|
+                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                    -- PRE_CLUSTERED_GROUP_BY[$$411, $$413]  |PARTITIONED|
+                                                                                                                                                                            {
+                                                                                                                                                                              -- AGGREGATE  |LOCAL|
+                                                                                                                                                                                -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                                                            }
+                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                        -- STABLE_SORT [$$411(ASC), $$413(ASC)]  |PARTITIONED|
+                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                -- HYBRID_HASH_JOIN [$$411, $$413][$$421, $$422]  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$411, $$413]  |PARTITIONED|
+                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$416][$$417]  |PARTITIONED|
+                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$417]  |PARTITIONED|
+                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                            -- UNNEST  |PARTITIONED|
+                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                -- PRE_CLUSTERED_GROUP_BY[$$427, $$428]  |PARTITIONED|
+                                                                                                                                                                                                        {
+                                                                                                                                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                                                                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                                                                                                                                              -- STREAM_SELECT  |LOCAL|
+                                                                                                                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                                                                                        }
+                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                    -- STABLE_SORT [$$427(ASC), $$428(ASC)]  |PARTITIONED|
+                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                            -- HYBRID_HASH_JOIN [$$427, $$428][$$437, $$438]  |PARTITIONED|
+                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$427, $$428]  |PARTITIONED|
+                                                                                                                                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                        -- HYBRID_HASH_JOIN [$$430][$$429]  |PARTITIONED|
+                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                      -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                                          -- HASH_PARTITION_EXCHANGE [$$429]  |PARTITIONED|
+                                                                                                                                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                              -- HASH_PARTITION_EXCHANGE [$$437, $$438]  |PARTITIONED|
+                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                      -- HYBRID_HASH_JOIN [$$445][$$444]  |PARTITIONED|
+                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                              -- HYBRID_HASH_JOIN [$$447][$$446]  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                          -- HYBRID_HASH_JOIN [$$447][$$450]  |PARTITIONED|
+                                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$416]  |PARTITIONED|
+                                                                                                                                                                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                                        -- DATASOURCE_SCAN (test.collection0)  |PARTITIONED|
+                                                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$450]  |PARTITIONED|
+                                                                                                                                                                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                                    -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$446]  |PARTITIONED|
+                                                                                                                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                                                                                                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                                                                          -- UNNEST  |UNPARTITIONED|
+                                                                                                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- UNNEST  |UNPARTITIONED|
+                                                                                                                                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$392]  |PARTITIONED|
+                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- REPLICATE  |PARTITIONED|
+                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                            -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                            -- HASH_PARTITION_EXCHANGE [$$218]  |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|
+                                                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                      -- DATASOURCE_SCAN (test.collection1)  |PARTITIONED|
+                                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- UNNEST  |UNPARTITIONED|
+                                                                                                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- UNNEST  |UNPARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
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
index 0f1879f..59b09a9 100644
--- 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
@@ -22,8 +22,8 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- BTREE_SEARCH (test.s.s)  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
-                  -- HASH_PARTITION_EXCHANGE [$$52]  |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/tpch/q12_shipping.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results_cbo/tpch/q12_shipping.plan
index 7609856..c1f2aff 100644
--- 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
@@ -3,13 +3,13 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$131]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$135]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
-              -- SORT_GROUP_BY[$$114]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$118]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
@@ -17,20 +17,20 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$118][$$122]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$118][$$124]  |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|
-                          -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$114][$$120]  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$122][$$126]  |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|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- STREAM_SELECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
@@ -38,6 +38,6 @@
                                             -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                  -- UNNEST  |UNPARTITIONED|
-                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- UNNEST  |UNPARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
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
index f21d402..b926894 100644
--- 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
@@ -9,13 +9,13 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- REPLICATE  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$131]  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$135]  |PARTITIONED|
                               {
                                 -- AGGREGATE  |LOCAL|
                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                               }
-                        -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
-                          -- SORT_GROUP_BY[$$114]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$118]  |PARTITIONED|
                                   {
                                     -- AGGREGATE  |LOCAL|
                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
@@ -23,20 +23,20 @@
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$118][$$122]  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$118][$$124]  |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|
-                                      -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$114][$$120]  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$122][$$126]  |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|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- STREAM_SELECT  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
                                                     -- STREAM_PROJECT  |PARTITIONED|
@@ -44,9 +44,9 @@
                                                         -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                              -- UNNEST  |UNPARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- UNNEST  |UNPARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
                 -- BROADCAST_EXCHANGE  |PARTITIONED|
                   -- AGGREGATE  |UNPARTITIONED|
                     -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
@@ -55,13 +55,13 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$131]  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$135]  |PARTITIONED|
                                         {
                                           -- AGGREGATE  |LOCAL|
                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                         }
-                                  -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
-                                    -- SORT_GROUP_BY[$$114]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$135]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$118]  |PARTITIONED|
                                             {
                                               -- AGGREGATE  |LOCAL|
                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
@@ -69,20 +69,20 @@
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$118][$$122]  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$118][$$124]  |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|
-                                                -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- HYBRID_HASH_JOIN [$$114][$$120]  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$122][$$126]  |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|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
                                                           -- STREAM_SELECT  |PARTITIONED|
                                                             -- ASSIGN  |PARTITIONED|
                                                               -- STREAM_PROJECT  |PARTITIONED|
@@ -90,6 +90,6 @@
                                                                   -- DATASOURCE_SCAN (tpch.LineItem)  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                        -- UNNEST  |UNPARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
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/profiled.xml b/asterixdb/asterix-app/src/test/resources/runtimets/profiled.xml
index f3209a1..d3664d4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/profiled.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/profiled.xml
@@ -38,5 +38,17 @@
         <output-dir compare="Text">full-scan-3</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="profile">
+      <compilation-unit name="sleep">
+        <parameter name="profile" value="timings" type="string"/>
+        <output-dir compare="Text">sleep</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="profile">
+      <compilation-unit name="non-unary-subplan">
+        <parameter name="profile" value="timings" type="string"/>
+        <output-dir compare="Text">non-unary-subplan</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
 </test-suite>
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/cache-residency/cache-residency.001.post.http
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/cache-residency/cache-residency.001.post.http
index 6afbccb..0edc12a 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/cache-residency/cache-residency.001.post.http
@@ -16,17 +16,5 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
-
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+/query/service
+--body={"statement": "from [1, 2] as v select v;"}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/cache-residency/cache-residency.002.post.http
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/cache-residency/cache-residency.002.post.http
index 6afbccb..9bf0936 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/cache-residency/cache-residency.002.post.http
@@ -16,17 +16,5 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
-
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+/query/service
+--body={"statement": "SELECT COUNT(*) FROM Metadata.`Dataset`"}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.1.ddl.sqlpp
new file mode 100644
index 0000000..601eb0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.1.ddl.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type addressType as closed {state: string, country: string, zip_code: int?};
+create type openType as {id: int};
+create type closedType as closed {id: int, list_f: [addressType]};
+
+create dataset openDs(openType) primary key id;
+create dataset closedDs(closedType) primary key id;
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.2.update.sqlpp
new file mode 100644
index 0000000..7e8e5ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.2.update.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
+
+insert into openDs([
+{"id": 1, "list_f": [ [1,2,1], [9999,3] ]},
+{"id": 2, "list_f": [ ["white","blue","magenta"], ["red", "black"] ]},
+{"id": 3, "list_f": [ 1 , 2 ]},
+{"id": 4, "list_f": [ {"state": "OH", "country": "US"} , {"state": "CA", "country": "US", "zip_code": 92863} ]},
+{"id": 5, "list_f": [ {"state": "OR", "country": "US", "zip_code": null} , {"state": "IL", "country": "US", "zip_code": 92863} ]},
+{"id": 6, "list_f": null},
+{"id": 7}
+]);
+
+insert into closedDs([
+{"id": 1, "list_f": [ {"state": "OH", "country": "US"} , {"state": "CA", "country": "US", "zip_code": 92863} ]},
+{"id": 2, "list_f": [ {"state": "OR", "country": "US", "zip_code": null} , {"state": "IL", "country": "US", "zip_code": 92863} ]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.3.query.sqlpp
new file mode 100644
index 0000000..eebfec8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.3.query.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use TinySocial;
+
+FROM [
+  {"id":1, "t1": (select array_binary_search(null, 1))},
+  {"id":2, "t2": (select array_binary_search(missing, 1))},
+  {"id":3, "t3": (select array_binary_search([], 1))},
+  {"id":4, "t4": (select array_binary_search([1,2,3,4,5,6,7], 4))},
+  {"id":5, "t5": (select array_binary_search([1,2,3,4,5,6,7], 1))},
+  {"id":6, "t6": (select array_binary_search([1,2,3,4,5,6,7], 7))},
+  {"id":7, "t7": (select array_binary_search(["a", "b", "c", "d", "e", "f", "g"], "c"))},
+  {"id":8, "t8": (select array_binary_search(["a", "b", "c", "d", "e", "f", "g"], "a"))},
+  {"id":9, "t9": (select array_binary_search(["a", "b", "c", "d", "e", "f", "g"], "g"))},
+  {"id":10, "t10": (select array_binary_search([1,2,3,4,5,6,7,8], 3.0))},
+  {"id":11, "t11": (select array_binary_search([1,2,3,4,5,6,7,8], 8.0))},
+  {"id":12, "t12": (select array_binary_search([1,2,3,4,5,6,7,8], 2.5))},
+  {"id":13, "t13": (select array_binary_search("not-an-array", 3))},
+  {"id":14, "t14": (select array_binary_search([ ["a", "b"], ["b", "c"], ["c", "d"], ["d", "e"] ], ["a", "b"]))},
+  {"id":15, "t15": (select array_binary_search([ ["a", "b"], ["b", "c"], ["c", "d"], ["d", "e"] ], ["d", "e"]))},
+  {"id":16, "t16": (from openDs select array_binary_search(list_f, [9999,3]) order by id)},
+  {"id":17, "t17": (from openDs select array_binary_search(list_f, {"state": "OH", "country": "US"}) order by id)},
+  {"id":18, "t18": (from closedDs select array_binary_search(list_f, {"state": "OH", "country": "US"}) order by id)},
+  {"id":19, "t19": (select array_binary_search([{"id": 1, "age": 34}, {"id": 2, "age": 29}, {"id": 3, "age": 90}, {"id": 4, "age": 10}], {"id": 2, "age": 29} ))},
+  {"id":20, "t20": (select array_binary_search([0,0,1,1,1,2,3,3,3,3,4,5,6,6,6,7], 3))},
+  {"id":21, "t21": (select array_binary_search(["a", "b", "b", "b", "c", "d", "e", "e", "f", "f", "f", "g", "h", "i", "j"], "f"))},
+  {"id":22, "t22": (select array_binary_search( {{1, 2, 3, 4, 5}}, 3))}
+] as d
+
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.4.ddl.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.4.ddl.sqlpp
index 6afbccb..3f8c8ec 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_binary_search/array_binary_search.4.ddl.sqlpp
@@ -16,17 +16,5 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.1.ddl.sqlpp
new file mode 100644
index 0000000..601eb0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.1.ddl.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type addressType as closed {state: string, country: string, zip_code: int?};
+create type openType as {id: int};
+create type closedType as closed {id: int, list_f: [addressType]};
+
+create dataset openDs(openType) primary key id;
+create dataset closedDs(closedType) primary key id;
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.2.update.sqlpp
new file mode 100644
index 0000000..7e8e5ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.2.update.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
+
+insert into openDs([
+{"id": 1, "list_f": [ [1,2,1], [9999,3] ]},
+{"id": 2, "list_f": [ ["white","blue","magenta"], ["red", "black"] ]},
+{"id": 3, "list_f": [ 1 , 2 ]},
+{"id": 4, "list_f": [ {"state": "OH", "country": "US"} , {"state": "CA", "country": "US", "zip_code": 92863} ]},
+{"id": 5, "list_f": [ {"state": "OR", "country": "US", "zip_code": null} , {"state": "IL", "country": "US", "zip_code": 92863} ]},
+{"id": 6, "list_f": null},
+{"id": 7}
+]);
+
+insert into closedDs([
+{"id": 1, "list_f": [ {"state": "OH", "country": "US"} , {"state": "CA", "country": "US", "zip_code": 92863} ]},
+{"id": 2, "list_f": [ {"state": "OR", "country": "US", "zip_code": null} , {"state": "IL", "country": "US", "zip_code": 92863} ]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.3.query.sqlpp
new file mode 100644
index 0000000..8659ed4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.3.query.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use TinySocial;
+
+FROM [
+    {"id":1, "t1" : (select array_move([0,1,2,3,4,5], 1, 3))},
+    {"id":2, "t2" : (select array_move([0,1,2,3,4,5], -1, -3))},
+    {"id":3, "t3" : (select array_move(["a", "b", "c", "d", "e"], 0, 2))},
+    {"id":4, "t4" : (select array_move(["a", "b", "c", "d", "e"], -2, -4))},
+    {"id":5, "t5" : (select array_move(null, 0, 1))},
+    {"id":6, "t6" : (select array_move([], 0, 1))},
+    {"id":7, "t7" : (select array_move([1,2,3,4,5], null, 1))},
+    {"id":8, "t8" : (select array_move([1,2,3,4,5], 0, null))},
+    {"id":9, "t9": (from openDs select array_move(list_f, 0, 1) order by id)},
+    {"id":10, "t10": (from openDs select array_move(list_f, -1, -2) order by id)},
+    {"id":11, "t11": (from openDs select array_move(list_f, null, 1) order by id)},
+    {"id":12, "t12": (from openDs select array_move(list_f, missing, 1) order by id)},
+    {"id":13, "t13": (from closedDs select array_move(list_f, 0, 1) order by id)},
+    {"id":14, "t14": (from closedDs select array_move(list_f, -1, -2) order by id)},
+    {"id":15, "t15": (from closedDs select array_move(list_f, null, 1) order by id)},
+    {"id":16, "t16": (from closedDs select array_move(list_f, missing, 1) order by id)},
+    {"id":17, "t17": (select array_move("not-an-array", 0, 1))},
+    {"id":18, "t18": (select array_move([1,2,3,4,5], 7, 8))},
+    {"id":19, "t19": (select array_move([1,2,3,4,5], "not-an-int", 1))},
+    {"id":20, "t20": (select array_move([1,2,3,4,5], 0, "not-an-int"))},
+    {"id":21, "t21": (select array_move(missing, 0, 1))},
+    {"id":22, "t22": (select array_move([1,2,3,4,5], missing, 1))},
+    {"id":23, "t23": (select array_move([1,2,3,4,5], 0, missing))},
+    {"id":24, "t24": (select array_move([1,2,3,4,5], 4, 0))},
+    {"id":25, "t25": (select array_move([1,2,3,4,5], 3, 1))},
+    {"id":26, "t26": (select array_move( {{1,2,3,4,5}}, 0, 3 ))}
+] as d
+
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.4.ddl.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.4.ddl.sqlpp
index 6afbccb..3f8c8ec 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_move/array_move.4.ddl.sqlpp
@@ -16,17 +16,5 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+drop  dataverse TinySocial;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.1.ddl.sqlpp
new file mode 100644
index 0000000..601eb0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.1.ddl.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+create type addressType as closed {state: string, country: string, zip_code: int?};
+create type openType as {id: int};
+create type closedType as closed {id: int, list_f: [addressType]};
+
+create dataset openDs(openType) primary key id;
+create dataset closedDs(closedType) primary key id;
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.2.update.sqlpp
new file mode 100644
index 0000000..7e8e5ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.2.update.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
+
+insert into openDs([
+{"id": 1, "list_f": [ [1,2,1], [9999,3] ]},
+{"id": 2, "list_f": [ ["white","blue","magenta"], ["red", "black"] ]},
+{"id": 3, "list_f": [ 1 , 2 ]},
+{"id": 4, "list_f": [ {"state": "OH", "country": "US"} , {"state": "CA", "country": "US", "zip_code": 92863} ]},
+{"id": 5, "list_f": [ {"state": "OR", "country": "US", "zip_code": null} , {"state": "IL", "country": "US", "zip_code": 92863} ]},
+{"id": 6, "list_f": null},
+{"id": 7}
+]);
+
+insert into closedDs([
+{"id": 1, "list_f": [ {"state": "OH", "country": "US"} , {"state": "CA", "country": "US", "zip_code": 92863} ]},
+{"id": 2, "list_f": [ {"state": "OR", "country": "US", "zip_code": null} , {"state": "IL", "country": "US", "zip_code": 92863} ]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.3.query.sqlpp
new file mode 100644
index 0000000..6689330
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.3.query.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use TinySocial;
+
+FROM [
+    {"id":1, "t1" : (select array_swap([0,1,2,3,4,5], 1, 3))},
+    {"id":2, "t2" : (select array_swap([0,1,2,3,4,5], -1, -3))},
+    {"id":3, "t3" : (select array_swap(["a", "b", "c", "d", "e"], 0, 2))},
+    {"id":4, "t4" : (select array_swap(["a", "b", "c", "d", "e"], -2, -4))},
+    {"id":5, "t5" : (select array_swap(null, 0, 1))},
+    {"id":6, "t6" : (select array_swap([], 0, 1))},
+    {"id":7, "t7" : (select array_swap([1,2,3,4,5], null, 1))},
+    {"id":8, "t8" : (select array_swap([1,2,3,4,5], 0, null))},
+    {"id":9, "t9": (from openDs select array_swap(list_f, 0, 1) order by id)},
+    {"id":10, "t10": (from openDs select array_swap(list_f, -1, -2) order by id)},
+    {"id":11, "t11": (from openDs select array_swap(list_f, null, 1) order by id)},
+    {"id":12, "t12": (from openDs select array_swap(list_f, missing, 1) order by id)},
+    {"id":13, "t13": (from closedDs select array_swap(list_f, 0, 1) order by id)},
+    {"id":14, "t14": (from closedDs select array_swap(list_f, -1, -2) order by id)},
+    {"id":15, "t15": (from closedDs select array_swap(list_f, null, 1) order by id)},
+    {"id":16, "t16": (from closedDs select array_swap(list_f, missing, 1) order by id)},
+    {"id":17, "t17" : (select array_swap("not-an-array", 0, 1))},
+    {"id":18, "t18" : (select array_swap([1,2,3,4,5], 7, 8))},
+    {"id":19, "t19" : (select array_swap([1,2,3,4,5], "not-an-int", 1))},
+    {"id":20, "t20" : (select array_swap([1,2,3,4,5], 0, "not-an-int"))},
+    {"id":21, "t21" : (select array_swap(missing, 0, 1))},
+    {"id":22, "t22" : (select array_swap([1,2,3,4,5], missing, 1))},
+    {"id":23, "t23" : (select array_swap([1,2,3,4,5], 0, missing))},
+    {"id":24, "t24": (select array_swap([1,2,3,4,5], 4, 0))},
+    {"id":25, "t25": (select array_swap([1,2,3,4,5], 3, 1))},
+    {"id":26, "t26": (select array_swap({{1,2,3,4,5}}, 1, 3))}
+] as d
+
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.4.ddl.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.4.ddl.sqlpp
index 6afbccb..3f8c8ec 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_swap/array_swap.4.ddl.sqlpp
@@ -16,17 +16,5 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+drop  dataverse TinySocial;
\ No newline at end of file
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 47eb439..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/join/nlj-outer/nlj-outer.00.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.00.ddl.sqlpp
new file mode 100644
index 0000000..a4d3984
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.00.ddl.sqlpp
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE tpch IF EXISTS;
+CREATE DATAVERSE tpch;
+
+USE tpch;
+
+CREATE TYPE CustomerType AS CLOSED {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+CREATE TYPE OrderType AS CLOSED {
+  o_orderkey : bigint,
+  o_custkey : bigint,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : bigint,
+  o_comment : string
+};
+
+CREATE DATASET Customer(CustomerType)
+PRIMARY KEY c_custkey;
+
+CREATE DATASET Orders(OrderType)
+PRIMARY KEY o_orderkey;
+
+
+CREATE INDEX customer_fk_nation ON Customer (c_nationkey);
+CREATE INDEX orders_fk_customer ON Orders (o_custkey);
+CREATE INDEX orders_orderdateIx ON Orders (o_orderdate);
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.01.update.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.01.update.sqlpp
index 6afbccb..3336fd7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.01.update.sqlpp
@@ -16,17 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+USE tpch;
 
-    void pause();
+LOAD DATASET Orders USING localfs (
+   (`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),
+   (`format`=`delimited-text`),
+   (`delimiter`=`|`)
+);
 
-    void resume();
-}
+LOAD DATASET Customer using localfs (
+   (`path`=`asterix_nc1://data/tpch0.001/customer.tbl`),
+   (`format`=`delimited-text`),
+   (`delimiter`=`|`)
+);
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.10.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.10.query.sqlpp
index 6afbccb..45fa904 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.10.query.sqlpp
@@ -16,17 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+USE tpch;
 
-    void pause();
-
-    void resume();
-}
+SELECT VALUE COUNT(*)
+FROM            Customer c
+LEFT OUTER JOIN Orders o
+ON  c.c_custkey  = o.o_custkey
+AND o.o_comment NOT LIKE '%special%requests%'
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.11.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.11.query.sqlpp
index 6afbccb..0d270a7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.11.query.sqlpp
@@ -16,17 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+USE tpch;
 
-    void pause();
-
-    void resume();
-}
+SELECT VALUE COUNT(*)
+FROM            Customer c 
+LEFT OUTER JOIN Orders o 
+ON  c.c_custkey /*+indexnl*/ = o.o_custkey
+AND o.o_comment NOT LIKE '%special%requests%'
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.20.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.20.query.sqlpp
index 6afbccb..5f5562b 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.20.query.sqlpp
@@ -16,17 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+USE tpch;
 
-    void pause();
-
-    void resume();
-}
+SELECT VALUE COUNT(*)
+FROM            Customer c
+LEFT OUTER JOIN Orders o
+ON  c.c_custkey  = o.o_custkey
+AND o.o_comment NOT LIKE '%special%requests%'
+AND o.o_totalprice > 100000
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.21.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.21.query.sqlpp
index 6afbccb..3057cd5 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.21.query.sqlpp
@@ -16,17 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+USE tpch;
 
-    void pause();
-
-    void resume();
-}
+SELECT VALUE COUNT(*)
+FROM            Customer c
+LEFT OUTER JOIN Orders o
+ON  c.c_custkey /*+indexnl*/ = o.o_custkey
+AND o.o_comment NOT LIKE '%special%requests%'
+AND o.o_totalprice > 100000
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.30.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.30.query.sqlpp
index 6afbccb..717da7f 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.30.query.sqlpp
@@ -16,17 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+USE tpch;
 
-    void pause();
-
-    void resume();
-}
+SELECT VALUE COUNT(*)
+FROM            Customer c
+LEFT OUTER JOIN Orders o
+ON  c.c_custkey  = o.o_custkey
+AND o.o_comment NOT LIKE '%special%requests%'
+AND (CASE WHEN o.o_totalprice > 100000.0 THEN TRUE ELSE FALSE END)
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.31.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.31.query.sqlpp
index 6afbccb..5ba532e 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.31.query.sqlpp
@@ -16,17 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+USE tpch;
 
-    void pause();
-
-    void resume();
-}
+SELECT VALUE COUNT(*)
+FROM            Customer c
+LEFT OUTER JOIN Orders o
+ON  c.c_custkey /*+indexnl*/ = o.o_custkey
+AND o.o_comment NOT LIKE '%special%requests%'
+AND (CASE WHEN o.o_totalprice > 100000.0 THEN TRUE ELSE FALSE END)
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.40.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.40.query.sqlpp
index 6afbccb..10475eb 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.40.query.sqlpp
@@ -16,17 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+USE tpch;
 
-    void pause();
-
-    void resume();
-}
+SELECT VALUE COUNT(*)
+FROM            Customer c
+LEFT OUTER JOIN Orders o
+ON  c.c_custkey  = o.o_custkey
+AND o.o_comment NOT LIKE '%special%requests%'
+AND (SELECT VALUE SUM(o.o_orderkey) FROM range(1, 10) x)[0] > 30000
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.41.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.41.query.sqlpp
index 6afbccb..d8b1908 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.41.query.sqlpp
@@ -16,17 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+USE tpch;
 
-    void pause();
-
-    void resume();
-}
+SELECT VALUE COUNT(*)
+FROM            Customer c
+LEFT OUTER JOIN Orders o
+ON  c.c_custkey /*+indexnl*/ = o.o_custkey
+AND o.o_comment NOT LIKE '%special%requests%'
+AND (SELECT VALUE SUM(o.o_orderkey) FROM range(1, 10) x)[0] > 30000
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.50.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.50.query.sqlpp
index 6afbccb..39f82b7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.50.query.sqlpp
@@ -16,17 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+USE tpch;
 
-    void pause();
-
-    void resume();
-}
+SELECT VALUE COUNT(*)
+FROM            Customer c
+LEFT OUTER JOIN Orders o
+ON  c.c_custkey  = o.o_custkey
+WHERE o.o_comment NOT LIKE '%special%requests%'
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.51.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.51.query.sqlpp
index 6afbccb..e15d56e 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/nlj-outer/nlj-outer.51.query.sqlpp
@@ -16,17 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+USE tpch;
 
-    void pause();
-
-    void resume();
-}
+SELECT VALUE COUNT(*)
+FROM            Customer c
+LEFT OUTER JOIN Orders o
+ON  c.c_custkey /*+indexnl*/ = o.o_custkey
+WHERE o.o_comment NOT LIKE '%special%requests%'
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/offset_without_limit/offset_without_limit.7.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/offset_without_limit/offset_without_limit.7.query.sqlpp
index 6afbccb..e0df538 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/offset_without_limit/offset_without_limit.7.query.sqlpp
@@ -16,17 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+/*
+ * Description     : Test that offset without limit is NOT pushed into a primary scan
+ * Expected Result : Success
+ */
 
-    void pause();
+use test;
 
-    void resume();
-}
+select x, count(*) as count from [{"id":10, "x":1},{"id":10, "x":2}] a group by x order by x offset 1;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.1.ddl.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.1.ddl.sqlpp
index 6afbccb..fbc5858 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.1.ddl.sqlpp
@@ -16,17 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+/*
+ * Description: This test case is to verify the fix for ASTERIXDB-2947
+ */
 
-    void pause();
+drop dataverse test if exists;
+create dataverse test;
 
-    void resume();
-}
+use test;
+
+create type dt1 as {id:int};
+create dataset collection0(dt1) primary key id;
+create dataset collection1(dt1) primary key id;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.2.update.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.2.update.sqlpp
index 6afbccb..2ab1a10 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.2.update.sqlpp
@@ -16,17 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+use test;
 
-    void pause();
+insert into collection0
+([
+  {"id": 1, "x_id":5, "y_id":5, "a":12, "b":20, "to_u":"aaaaa", "posi":"a", "u":"col0"}
+]);
 
-    void resume();
-}
+insert into collection1
+([
+  {"id": 1, "x_id":5, "y_id":5, "a":12, "b":20, "to_u":"aaaaa", "posi":"a", "u":"col1" }
+]);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.3.query.sqlpp
new file mode 100644
index 0000000..09fdcda
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.3.query.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: This test case is to verify the fix for ASTERIXDB-3316
+ */
+
+use test;
+
+SELECT DISTINCT T1.*,
+        (SELECT H.u
+                   FROM collection1 H
+                  WHERE H.y_id = T1.x_id
+                    AND H.a IN [12, 66, 67, 13, 26]
+                    AND H.to_u = 'aaaaa'
+                    AND H.b in
+                        (SELECT value MAX(L.b)
+                           FROM collection1 L
+                          WHERE L.y_id = T1.x_id
+                            AND L.a IN [12, 66, 67, 13, 26]
+                            AND L.to_u = 'aaaaa')
+                            ) AS sub_query1,
+        (SELECT H.u
+                   FROM collection1 H
+                  WHERE  H.y_id = T1.x_id
+                    AND H.a IN [12, 66, 67, 13]
+                    AND H.posi IN ['a','b']
+                    AND H.b in
+                        (SELECT value MAX(L.b)
+                           FROM collection1 L
+                          WHERE L.y_id = T1.x_id
+                            AND L.posi IN ['a','b'])
+                                  ) AS sub_query2
+FROM (SELECT T0.x_id
+        FROM collection0 T0
+) T1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
index 8532fe9..77fc7c5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
@@ -124,6 +124,10 @@
     <compilation-unit name="object_concat">
       <output-dir compare="Text">object_concat</output-dir>
       <expected-warn>ASX0013: Duplicate field name 'v'</expected-warn>
+      <expected-warn>ASX0013: Duplicate field name 'f'</expected-warn>
+      <expected-warn>ASX0013: Duplicate field name 'id'</expected-warn>
+      <expected-warn>ASX0013: Duplicate field name 'id'</expected-warn>
+      <expected-warn>ASX0013: Duplicate field name 'dup'</expected-warn>
     </compilation-unit>
   </test-case>
   <test-case FilePath="objects">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.5.ddl.sqlpp
new file mode 100644
index 0000000..2eefbe8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.5.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       UsersBaseTypeClosed AS CLOSED { _id: int };
+CREATE TYPE       UsersBaseTypeOpen AS { _id: int };
+CREATE TYPE       UsersFriendsTypeClosed AS CLOSED {
+  _id: int,
+  best_friend: UsersBaseTypeClosed,
+  friends: [UsersBaseTypeClosed]
+};
+CREATE TYPE       UsersFriendsTypeOpen AS { _id: int };
+
+CREATE DATASET    UsersClosed (UsersFriendsTypeClosed) PRIMARY KEY _id;
+CREATE DATASET    UsersOpen (UsersFriendsTypeOpen) PRIMARY KEY _id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.6.update.sqlpp
new file mode 100644
index 0000000..bda29eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.6.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE               TestDataverse;
+
+INSERT INTO       UsersClosed [
+  { "_id": 1, "best_friend": { "_id": 1 }, "friends": [] },
+  { "_id": 2, "best_friend": { "_id": 1 }, "friends": [{ "_id": 1 }] },
+  { "_id": 3, "best_friend": { "_id": 2 }, "friends": [{ "_id": 1 }, { "_id": 2 }] }
+];
+INSERT INTO       UsersOpen [
+  { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [] },
+  { "_id": 5, "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [{ "_id": 4 }] },
+  { "_id": 6, "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [{ "_id": 4 }, { "_id": 5 }] }
+];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.7.query.sqlpp
new file mode 100644
index 0000000..71f6974
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.7.query.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// For OBJECT_ADD.
+USE     TestDataverse;
+LET     openUValues = (
+          FROM    UsersOpen U2
+          WHERE   U2._id = 4
+          SELECT  VALUE U2
+        ),
+        closedUValues = (
+          FROM    UsersClosed U1
+          WHERE   U1._id = 1
+          SELECT  VALUE U1
+        )
+SELECT  VALUE {
+  // New constant field into closed and open record.
+  "t1c": ( FROM UsersClosed U SELECT VALUE OBJECT_ADD(U, "name", "John") ORDER BY U._id ),
+  "t1o": ( FROM UsersOpen U SELECT VALUE OBJECT_ADD(U, "name", "John") ORDER BY U._id ),
+
+  // New record field from open record into closed and open record.
+  "t2c": ( FROM UsersClosed U SELECT VALUE OBJECT_ADD(U, "newFriend", openUValues[0]) ORDER BY U._id ) ,
+  "t2o": ( FROM UsersOpen U SELECT VALUE OBJECT_ADD(U, "newFriend", openUValues[0]) ORDER BY U._id ) ,
+
+  // New record field from closed record into closed and open record.
+  "t3c": ( FROM UsersClosed U SELECT VALUE OBJECT_ADD(U, "newFriend", closedUValues[0]) ORDER BY U._id ),
+  "t3o": ( FROM UsersOpen U SELECT VALUE OBJECT_ADD(U, "newFriend", closedUValues[0]) ORDER BY U._id ),
+
+  // New list field from open dataset into closed and open record.
+  "t4c": ( FROM UsersClosed U SELECT VALUE OBJECT_ADD(U, "newFriends", openUValues) ORDER BY U._id ),
+  "t4o": ( FROM UsersOpen U SELECT VALUE OBJECT_ADD(U, "newFriends", openUValues) ORDER BY U._id ),
+
+  // New list field from closed dataset into closed and open record.
+  "t5c": ( FROM UsersClosed U SELECT VALUE OBJECT_ADD(U, "newFriends", closedUValues) ORDER BY U._id ),
+  "t5o": ( FROM UsersOpen U SELECT VALUE OBJECT_ADD(U, "newFriends", closedUValues) ORDER BY U._id )
+};
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.5.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.5.query.sqlpp
index 6afbccb..1b61e66 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.5.query.sqlpp
@@ -16,17 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+// test that object_concat() issues a warning when encountering a duplicate field
 
-    void pause();
+// requesttype=application/json
+// param max-warnings:json=10
 
-    void resume();
-}
+SET `import-private-functions` `true`;
+WITH t AS ([{"id": 1, "f": 3}, {"id": 2, "f": 4}])
+SELECT OBJECT_CONCAT_STRICT(t) AS res;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.6.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.6.query.sqlpp
index 6afbccb..8ef2ebbb 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.6.query.sqlpp
@@ -16,17 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+// requesttype=application/json
+// param max-warnings:json=10
 
-    void pause();
-
-    void resume();
-}
+WITH a AS (SELECT VALUE x FROM [{"id": 1, "a1": 3, "dup": 0}] AS x),
+b AS (SELECT VALUE y FROM [{"id": 1, "b1": 3, "dup": 5}] AS y)
+FROM a, b
+WHERE a.id = b.id
+SELECT a.*, b.*
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.5.ddl.sqlpp
new file mode 100644
index 0000000..2eefbe8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.5.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       UsersBaseTypeClosed AS CLOSED { _id: int };
+CREATE TYPE       UsersBaseTypeOpen AS { _id: int };
+CREATE TYPE       UsersFriendsTypeClosed AS CLOSED {
+  _id: int,
+  best_friend: UsersBaseTypeClosed,
+  friends: [UsersBaseTypeClosed]
+};
+CREATE TYPE       UsersFriendsTypeOpen AS { _id: int };
+
+CREATE DATASET    UsersClosed (UsersFriendsTypeClosed) PRIMARY KEY _id;
+CREATE DATASET    UsersOpen (UsersFriendsTypeOpen) PRIMARY KEY _id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.6.update.sqlpp
new file mode 100644
index 0000000..bda29eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.6.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE               TestDataverse;
+
+INSERT INTO       UsersClosed [
+  { "_id": 1, "best_friend": { "_id": 1 }, "friends": [] },
+  { "_id": 2, "best_friend": { "_id": 1 }, "friends": [{ "_id": 1 }] },
+  { "_id": 3, "best_friend": { "_id": 2 }, "friends": [{ "_id": 1 }, { "_id": 2 }] }
+];
+INSERT INTO       UsersOpen [
+  { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [] },
+  { "_id": 5, "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [{ "_id": 4 }] },
+  { "_id": 6, "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [{ "_id": 4 }, { "_id": 5 }] }
+];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.7.query.sqlpp
new file mode 100644
index 0000000..52943a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.7.query.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// For OBJECT_PUT.
+USE     TestDataverse;
+LET     openUValues = (
+          FROM    UsersOpen U2
+          WHERE   U2._id = 4
+          SELECT  VALUE U2
+        ),
+        closedUValues = (
+          FROM    UsersClosed U1
+          WHERE   U1._id = 1
+          SELECT  VALUE U1
+        )
+SELECT  VALUE {
+  // New constant field into closed and open record.
+  "t1c": ( FROM UsersClosed U SELECT VALUE OBJECT_PUT(U, "name", "John") ORDER BY U._id ),
+  "t1o": ( FROM UsersOpen U SELECT VALUE OBJECT_PUT(U, "name", "John") ORDER BY U._id ),
+
+  // New record field from open record into closed and open record.
+  "t2c": ( FROM UsersClosed U SELECT VALUE OBJECT_PUT(U, "newFriend", openUValues[0]) ORDER BY U._id ) ,
+  "t2o": ( FROM UsersOpen U SELECT VALUE OBJECT_PUT(U, "newFriend", openUValues[0]) ORDER BY U._id ) ,
+
+  // New record field from closed record into closed and open record.
+  "t3c": ( FROM UsersClosed U SELECT VALUE OBJECT_PUT(U, "newFriend", closedUValues[0]) ORDER BY U._id ),
+  "t3o": ( FROM UsersOpen U SELECT VALUE OBJECT_PUT(U, "newFriend", closedUValues[0]) ORDER BY U._id ),
+
+  // New list field from open dataset into closed and open record.
+  "t4c": ( FROM UsersClosed U SELECT VALUE OBJECT_PUT(U, "newFriends", openUValues) ORDER BY U._id ),
+  "t4o": ( FROM UsersOpen U SELECT VALUE OBJECT_PUT(U, "newFriends", openUValues) ORDER BY U._id ),
+
+  // New list field from closed dataset into closed and open record.
+  "t5c": ( FROM UsersClosed U SELECT VALUE OBJECT_PUT(U, "newFriends", closedUValues) ORDER BY U._id ),
+  "t5o": ( FROM UsersOpen U SELECT VALUE OBJECT_PUT(U, "newFriends", closedUValues) ORDER BY U._id ),
+
+  // Old field of open type into closed record (we should overwrite).
+  "t6c": ( FROM   UsersClosed U
+           LET    newFriends = [
+             { "_id": 8 }, { "_id": 9 }
+           ]
+           SELECT VALUE OBJECT_PUT(U, "friends", newFriends)
+           ORDER BY U._id )
+};
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/non-unary-subplan/non-unary-subplan.1.ddl.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/non-unary-subplan/non-unary-subplan.1.ddl.sqlpp
index 6afbccb..c1ce4a3 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/non-unary-subplan/non-unary-subplan.1.ddl.sqlpp
@@ -16,17 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+drop dataverse test if exists;
+create dataverse test;
+use test;
 
-    void pause();
+create type TType as open
+{ id: bigint };
 
-    void resume();
-}
+create dataset TData (TType) primary key id;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/non-unary-subplan/non-unary-subplan.2.update.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/non-unary-subplan/non-unary-subplan.2.update.sqlpp
index 6afbccb..3ce9554 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/non-unary-subplan/non-unary-subplan.2.update.sqlpp
@@ -16,17 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+use test;
 
-    void pause();
-
-    void resume();
-}
+insert into TData ( [
+{'id':1, 'x':1, 'f':19},
+{'id':2, 'x':2, 'f':12},
+{'id':3, 'x':1, 'f':10},
+{'id':4, 'x':2, 'f':17},
+{'id':5, 'x':1, 'f':12}
+]);
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/non-unary-subplan/non-unary-subplan.3.profile.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/non-unary-subplan/non-unary-subplan.3.profile.sqlpp
index 6afbccb..4fa986c 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/non-unary-subplan/non-unary-subplan.3.profile.sqlpp
@@ -16,17 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+-- compareunorderedarray=true
+use test;
 
-    void pause();
-
-    void resume();
-}
+select value
+array_sum((
+   select value a.f
+   from g as p
+   union all
+   select value a.f
+   from g as w
+))
+from TData as a
+group by a.x as x group as g
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.1.ddl.sqlpp
new file mode 100644
index 0000000..aedce7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.1.ddl.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Testing that "off" is supplied in request parameter "profile".
+ * Expected Res : Success with expected result not having "profile" field.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type test.AddressType as
+{
+  number : bigint,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : bigint,
+  name : string,
+  age : bigint?,
+  address : AddressType?,
+  lastorder : {
+      oid : bigint,
+      total : float
+  }
+};
+
+create type test.OrderType as
+{
+  oid : bigint,
+  cid : bigint,
+  orderstatus : string,
+  orderpriority : string,
+  clerk : string,
+  total : float
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+create  dataset Orders(OrderType) primary key oid;
+
+create dataset Customers2(CustomerType) primary key cid;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.2.update.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.2.update.sqlpp
index 6afbccb..07af22e 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.2.update.sqlpp
@@ -16,17 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+use test;
 
-    void pause();
+load dataset Customers using localfs
+  ((`path`=`asterix_nc1://data/custord-tiny/customer-tiny-neg.adm`),
+  (`format`=`adm`));
 
-    void resume();
-}
+load dataset Orders using localfs
+  ((`path`=`asterix_nc1://data/custord-tiny/order-tiny.adm`),
+  (`format`=`adm`));
+
+load dataset Customers2 using localfs
+  ((`path`=`asterix_nc1://data/custord-tiny/customer-tiny.adm`),
+  (`format`=`adm`));
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.3.profile.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.3.profile.sqlpp
index 6afbccb..43f8e4a 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.3.profile.sqlpp
@@ -16,17 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+-- compareunorderedarray=true
+use test;
 
-    void pause();
-
-    void resume();
-}
+select sleep(c.cid,500) from Customers c;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.4.profile.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.4.profile.sqlpp
index 6afbccb..ea35b89 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.4.profile.sqlpp
@@ -16,17 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+-- compareunorderedarray=true
+USE test;
 
-    void pause();
-
-    void resume();
-}
+SELECT count(*) AS customers, city
+FROM Customers c
+WHERE c.age < 65
+GROUP BY sleep(c.address.city, 1000) AS city;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.5.profile.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.5.profile.sqlpp
index 6afbccb..6c35376 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.5.profile.sqlpp
@@ -16,17 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+-- compareunorderedarray=true
+USE test;
 
-    void pause();
-
-    void resume();
-}
+SELECT count(*) AS customers, city
+FROM Customers c
+WHERE c.age <65
+GROUP BY c.address.city
+ORDER BY sleep(city,1700);
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.90.ddl.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.90.ddl.sqlpp
index 6afbccb..f12a2b7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/profile/sleep/sleep.90.ddl.sqlpp
@@ -16,17 +16,5 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+drop dataverse test;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.1.ddl.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.1.ddl.sqlpp
index 6afbccb..8fc7172 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.1.ddl.sqlpp
@@ -16,17 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+DROP DATAVERSE ComplexExclude IF EXISTS;
+CREATE DATAVERSE ComplexExclude;
+USE ComplexExclude;
 
-    void pause();
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
 
-    void resume();
-}
+CREATE VIEW UsersWithoutPII AS
+    FROM ComplexExclude.Users U
+    SELECT U.* EXCLUDE phones, address, ssn;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.10.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.10.query.sqlpp
index 6afbccb..67d91fb 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.10.query.sqlpp
@@ -16,17 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+//         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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.11.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.11.query.sqlpp
index 6afbccb..099c6c5 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.11.query.sqlpp
@@ -16,17 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+//     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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.12.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.12.query.sqlpp
index 6afbccb..6b5ac14 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.12.query.sqlpp
@@ -16,17 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+//       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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.13.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.13.query.sqlpp
index 6afbccb..46bac5d 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.13.query.sqlpp
@@ -16,17 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+//       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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.2.update.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.2.update.sqlpp
index 6afbccb..4c770f6 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.2.update.sqlpp
@@ -16,17 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+USE ComplexExclude;
 
-    void pause();
-
-    void resume();
-}
+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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.3.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.3.query.sqlpp
index 6afbccb..bc01577 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.3.query.sqlpp
@@ -16,17 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+//       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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.4.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.4.query.sqlpp
index 6afbccb..6aa8a33 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.4.query.sqlpp
@@ -16,17 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+//       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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.5.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.5.query.sqlpp
index 6afbccb..8ef1df6 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.5.query.sqlpp
@@ -16,17 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+//       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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.6.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.6.query.sqlpp
index 6afbccb..8e0b660 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.6.query.sqlpp
@@ -16,17 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+//       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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.7.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.7.query.sqlpp
index 6afbccb..ba6588e 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.7.query.sqlpp
@@ -16,17 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+//       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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.8.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.8.query.sqlpp
index 6afbccb..437c7af 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.8.query.sqlpp
@@ -16,17 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+//         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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.9.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.9.query.sqlpp
index 6afbccb..ff507a3 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/complex-exclude/complex-exclude.9.query.sqlpp
@@ -16,17 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+//       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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.1.ddl.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.1.ddl.sqlpp
index 6afbccb..7c54e82 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.1.ddl.sqlpp
@@ -16,17 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+DROP DATAVERSE ExcludeNegative IF EXISTS;
+CREATE DATAVERSE ExcludeNegative;
+USE ExcludeNegative;
 
-    void pause();
-
-    void resume();
-}
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.2.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.2.query.sqlpp
index 6afbccb..8e8c021 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.2.query.sqlpp
@@ -16,17 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+//       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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.3.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.3.query.sqlpp
index 6afbccb..eee1023 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/exclude-negative/exclude-negative.3.query.sqlpp
@@ -16,17 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+//       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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.2.update.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.2.update.sqlpp
index 6afbccb..a60ef4d 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.2.update.sqlpp
@@ -16,17 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
-
-    void pause();
-
-    void resume();
-}
+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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.3.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.3.query.sqlpp
index 6afbccb..4d7489a 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.3.query.sqlpp
@@ -16,17 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+FROM     TinySocial.TweetMessages TM
 
-    void pause();
+         // SELECT-var.*.
+SELECT   TM.*
 
-    void resume();
-}
+         // 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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.4.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.4.query.sqlpp
index 6afbccb..06c6df0 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.4.query.sqlpp
@@ -16,17 +16,20 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+FROM     TinySocial.TweetMessages TM,
+         TinySocial.TwitterUsers TU
 
-    void pause();
+WHERE    TM.user.`screen-name` = TU.`screen-name`
 
-    void resume();
-}
+         // 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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.5.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.5.query.sqlpp
index 6afbccb..d41b3a7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.5.query.sqlpp
@@ -16,17 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+FROM     TinySocial.TweetMessages TM1
+SELECT   TM1.*
+EXCLUDE   user.does_not_exist,
+         user.lang,
+         `send-time`,
+         does_not_exist
 
-    void pause();
+UNION ALL
 
-    void resume();
-}
+FROM     TinySocial.TweetMessages TM2
+SELECT   TM2.*
+EXCLUDE   user.lang
+
+ORDER BY tweetid, `send-time` NULLS FIRST;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.6.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.6.query.sqlpp
index 6afbccb..e5453ef 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/select-exclude/tiny-social/tiny-social.6.query.sqlpp
@@ -16,17 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+FROM     TinySocial.TweetMessages TM
 
-    void pause();
+         // SELECT-* (single-variable rule case).
+SELECT   *
 
-    void resume();
-}
+         // 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/union/union_type_cast/union_type_cast.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.1.ddl.sqlpp
new file mode 100644
index 0000000..2a616ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.1.ddl.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE TinySocial IF EXISTS;
+CREATE DATAVERSE TinySocial;
+
+USE TinySocial;
+
+CREATE TYPE EmploymentType AS CLOSED
+{
+  `organization-name` : string,
+  `start-date` : date,
+  `end-date` : date?
+};
+
+CREATE TYPE FacebookUserType AS CLOSED {
+  id : bigint,
+  alias : string,
+  name : string,
+  `user-since` : datetime,
+  `friend-ids` : {{bigint}},
+  employment : [EmploymentType]
+};
+
+CREATE TYPE FacebookMessageType AS CLOSED {
+  `message-id` : bigint,
+  `author-id` : bigint,
+  `in-response-to` : bigint?,
+  `sender-location` : point?,
+  message : string
+};
+
+CREATE DATASET FacebookUsers(FacebookUserType) PRIMARY KEY id;
+
+CREATE DATASET FacebookMessages(FacebookMessageType) PRIMARY KEY `message-id`;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.2.update.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.2.update.sqlpp
index 6afbccb..3d785ad 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.2.update.sqlpp
@@ -16,17 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+USE TinySocial;
 
-    void pause();
+LOAD DATASET FacebookUsers using localfs (("path"="asterix_nc1://data/tinysocial/fbu.adm"),("format"="adm"));
 
-    void resume();
-}
+LOAD DATASET FacebookMessages using localfs (("path"="asterix_nc1://data/tinysocial/fbm.adm"),("format"="adm"));
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.3.query.sqlpp
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.3.query.sqlpp
index 6afbccb..003ec26 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.3.query.sqlpp
@@ -16,17 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+USE TinySocial;
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
 
-    void pause();
 
-    void resume();
-}
+    SELECT "message1" ds, s.`message-id` id, object_remove(s, "in-response-to") no_in_response_to
+    FROM FacebookMessages AS s
+UNION ALL
+    SELECT "user" ds, t.id id, t user
+    FROM FacebookUsers t
+UNION ALL
+    SELECT "message2" ds, s.`message-id` id, object_remove(s, "author-id") no_author_id
+    FROM FacebookMessages s
+ORDER BY id, ds;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.4.query.sqlpp
similarity index 63%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.4.query.sqlpp
index 6afbccb..cc63b57 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_type_cast/union_type_cast.4.query.sqlpp
@@ -16,17 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+USE TinySocial;
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
 
-    void pause();
+-- To prevent the plan from changing
+SET `compiler.parallelism` "0";
 
-    void resume();
-}
+EXPLAIN
+    SELECT s.`message-id` id, object_remove(s, "in-response-to") no_in_response_to
+    FROM FacebookMessages AS s
+UNION ALL
+    SELECT t.id id, t user
+    FROM FacebookUsers t
+UNION ALL
+    SELECT s.`message-id` id, object_remove(s, "author-id") no_author_id
+    FROM FacebookMessages s
+ORDER BY id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cache-residency/cache-residency.001.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cache-residency/cache-residency.001.regexjson
new file mode 100644
index 0000000..173a9a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cache-residency/cache-residency.001.regexjson
@@ -0,0 +1,19 @@
+{
+	"requestID": "R{.*}",
+	"signature": {
+		"*": "*"
+	},
+	"type": "application/x-adm",
+	"results": [ "{ \"v\": 1 }\n", "{ \"v\": 2 }\n" ],
+	"plans": "R{.*}",
+	"status": "success",
+	"metrics": {
+		"elapsedTime": "R{.*}",
+		"executionTime": "R{.*}",
+		"compileTime": "R{.*}",
+		"queueWaitTime": "R{.*}",
+		"resultCount": 2,
+		"resultSize": 32,
+		"processedObjects": 0
+	}
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cache-residency/cache-residency.002.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cache-residency/cache-residency.002.regexjson
new file mode 100644
index 0000000..8f5cf76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cache-residency/cache-residency.002.regexjson
@@ -0,0 +1,20 @@
+{
+	"requestID": "R{.*}",
+	"signature": {
+		"*": "*"
+	},
+	"type": "application/x-adm",
+	"results": [ "{ \"$1\": 17 }\n" ],
+	"plans": "R{.*}",
+	"status": "success",
+	"metrics": {
+		"elapsedTime": "R{.*}",
+		"executionTime": "R{.*}",
+		"compileTime": "R{.*}",
+		"queueWaitTime": "R{.*}",
+		"resultCount": 1,
+		"resultSize": 18,
+		"processedObjects": 17,
+		"bufferCacheHitRatio": "100.00%"
+	}
+}
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 cb10dd2..03cbc23 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,6 +10,7 @@
     "active\.suspend\.timeout" : 3600,
     "azure.request.timeout" : 120,
     "compiler\.arrayindex" : true,
+    "compiler.batch.lookup" : false,
     "compiler.cbo" : false,
     "compiler\.external\.field\.pushdown" : true,
     "compiler.forcejoinorder" : false,
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 59d595b..e4f41d5 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,6 +10,7 @@
     "active\.suspend\.timeout" : 3600,
     "azure.request.timeout" : 120,
     "compiler\.arrayindex" : true,
+    "compiler.batch.lookup" : false,
     "compiler.cbo" : false,
     "compiler\.external\.field\.pushdown" : true,
     "compiler.forcejoinorder" : false,
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 56b413d..2c26e75 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,6 +10,7 @@
     "active\.suspend\.timeout" : 3600,
     "azure.request.timeout" : 120,
     "compiler\.arrayindex" : true,
+    "compiler.batch.lookup" : false,
     "compiler.cbo" : false,
     "compiler\.external\.field\.pushdown" : true,
     "compiler.forcejoinorder" : false,
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..823515b 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,9 +6,12 @@
 	"metrics": {
 		"elapsedTime": "R{.*}",
 		"executionTime": "R{.*}",
+		"compileTime": "R{.*}",
+		"queueWaitTime": "R{.*}",
 		"resultCount": 0,
 		"resultSize": 0,
 		"processedObjects": 0,
+		"bufferCacheHitRatio": "0.00%",
 		"errorCount": 1
 	}
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_binary_search/array_binary_search.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_binary_search/array_binary_search.3.adm
new file mode 100644
index 0000000..ae6f41f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_binary_search/array_binary_search.3.adm
@@ -0,0 +1,23 @@
+{ "id": 1, "t1": [ { "$1": null } ] }
+{ "id": 2, "t2": [ {  } ] }
+{ "id": 3, "t3": [ { "$3": -1 } ] }
+{ "id": 4, "t4": [ { "$4": 3 } ] }
+{ "id": 5, "t5": [ { "$5": 0 } ] }
+{ "id": 6, "t6": [ { "$6": 6 } ] }
+{ "id": 7, "t7": [ { "$7": 2 } ] }
+{ "id": 8, "t8": [ { "$8": 0 } ] }
+{ "id": 9, "t9": [ { "$9": 6 } ] }
+{ "id": 10, "t10": [ { "$10": 2 } ] }
+{ "id": 11, "t11": [ { "$11": 7 } ] }
+{ "id": 12, "t12": [ { "$12": -1 } ] }
+{ "id": 13, "t13": [ { "$13": null } ] }
+{ "id": 14, "t14": [ { "$14": 0 } ] }
+{ "id": 15, "t15": [ { "$15": 3 } ] }
+{ "id": 16, "t16": [ { "$16": 1 }, { "$16": -1 }, { "$16": -1 }, { "$16": -1 }, { "$16": -1 }, { "$16": null }, {  } ] }
+{ "id": 17, "t17": [ { "$17": -1 }, { "$17": -1 }, { "$17": -1 }, { "$17": 0 }, { "$17": -1 }, { "$17": null }, {  } ] }
+{ "id": 18, "t18": [ { "$18": 0 }, { "$18": -1 } ] }
+{ "id": 19, "t19": [ { "$19": 1 } ] }
+{ "id": 20, "t20": [ { "$20": 6 } ] }
+{ "id": 21, "t21": [ { "$21": 8 } ] }
+{ "id": 22, "t22": [ { "$22": null } ] }
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_move/array_move.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_move/array_move.3.adm
new file mode 100644
index 0000000..f8057c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_move/array_move.3.adm
@@ -0,0 +1,26 @@
+{ "id": 1, "t1": [ { "$1": [ 0, 2, 3, 1, 4, 5 ] } ] }
+{ "id": 2, "t2": [ { "$2": [ 0, 1, 2, 5, 3, 4 ] } ] }
+{ "id": 3, "t3": [ { "$3": [ "b", "c", "a", "d", "e" ] } ] }
+{ "id": 4, "t4": [ { "$4": [ "a", "d", "b", "c", "e" ] } ] }
+{ "id": 5, "t5": [ { "$5": null } ] }
+{ "id": 6, "t6": [ { "$6": null } ] }
+{ "id": 7, "t7": [ { "$7": null } ] }
+{ "id": 8, "t8": [ { "$8": null } ] }
+{ "id": 9, "t9": [ { "$9": [ [ 9999, 3 ], [ 1, 2, 1 ] ] }, { "$9": [ [ "red", "black" ], [ "white", "blue", "magenta" ] ] }, { "$9": [ 2, 1 ] }, { "$9": [ { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] }, { "$9": [ { "state": "IL", "country": "US", "zip_code": 92863 }, { "state": "OR", "country": "US", "zip_code": null } ] }, { "$9": null }, {  } ] }
+{ "id": 10, "t10": [ { "$10": [ [ 9999, 3 ], [ 1, 2, 1 ] ] }, { "$10": [ [ "red", "black" ], [ "white", "blue", "magenta" ] ] }, { "$10": [ 2, 1 ] }, { "$10": [ { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] }, { "$10": [ { "state": "IL", "country": "US", "zip_code": 92863 }, { "state": "OR", "country": "US", "zip_code": null } ] }, { "$10": null }, {  } ] }
+{ "id": 11, "t11": [ { "$11": null }, { "$11": null }, { "$11": null }, { "$11": null }, { "$11": null }, { "$11": null }, {  } ] }
+{ "id": 12, "t12": [ {  }, {  }, {  }, {  }, {  }, {  }, {  } ] }
+{ "id": 13, "t13": [ { "$13": [ { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] }, { "$13": [ { "state": "IL", "country": "US", "zip_code": 92863 }, { "state": "OR", "country": "US", "zip_code": null } ] } ] }
+{ "id": 14, "t14": [ { "$14": [ { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] }, { "$14": [ { "state": "IL", "country": "US", "zip_code": 92863 }, { "state": "OR", "country": "US", "zip_code": null } ] } ] }
+{ "id": 15, "t15": [ { "$15": null }, { "$15": null } ] }
+{ "id": 16, "t16": [ {  }, {  } ] }
+{ "id": 17, "t17": [ { "$17": null } ] }
+{ "id": 18, "t18": [ { "$18": null } ] }
+{ "id": 19, "t19": [ { "$19": null } ] }
+{ "id": 20, "t20": [ { "$20": null } ] }
+{ "id": 21, "t21": [ {  } ] }
+{ "id": 22, "t22": [ {  } ] }
+{ "id": 23, "t23": [ {  } ] }
+{ "id": 24, "t24": [ { "$24": [ 5, 1, 2, 3, 4 ] } ] }
+{ "id": 25, "t25": [ { "$25": [ 1, 4, 2, 3, 5 ] } ] }
+{ "id": 26, "t26": [ { "$26": null } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_swap/array_swap.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_swap/array_swap.3.adm
new file mode 100644
index 0000000..97ccdc4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_swap/array_swap.3.adm
@@ -0,0 +1,26 @@
+{ "id": 1, "t1": [ { "$1": [ 0, 3, 2, 1, 4, 5 ] } ] }
+{ "id": 2, "t2": [ { "$2": [ 0, 1, 2, 5, 4, 3 ] } ] }
+{ "id": 3, "t3": [ { "$3": [ "c", "b", "a", "d", "e" ] } ] }
+{ "id": 4, "t4": [ { "$4": [ "a", "d", "c", "b", "e" ] } ] }
+{ "id": 5, "t5": [ { "$5": null } ] }
+{ "id": 6, "t6": [ { "$6": null } ] }
+{ "id": 7, "t7": [ { "$7": null } ] }
+{ "id": 8, "t8": [ { "$8": null } ] }
+{ "id": 9, "t9": [ { "$9": [ [ 9999, 3 ], [ 1, 2, 1 ] ] }, { "$9": [ [ "red", "black" ], [ "white", "blue", "magenta" ] ] }, { "$9": [ 2, 1 ] }, { "$9": [ { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] }, { "$9": [ { "state": "IL", "country": "US", "zip_code": 92863 }, { "state": "OR", "country": "US", "zip_code": null } ] }, { "$9": null }, {  } ] }
+{ "id": 10, "t10": [ { "$10": [ [ 9999, 3 ], [ 1, 2, 1 ] ] }, { "$10": [ [ "red", "black" ], [ "white", "blue", "magenta" ] ] }, { "$10": [ 2, 1 ] }, { "$10": [ { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] }, { "$10": [ { "state": "IL", "country": "US", "zip_code": 92863 }, { "state": "OR", "country": "US", "zip_code": null } ] }, { "$10": null }, {  } ] }
+{ "id": 11, "t11": [ { "$11": null }, { "$11": null }, { "$11": null }, { "$11": null }, { "$11": null }, { "$11": null }, {  } ] }
+{ "id": 12, "t12": [ {  }, {  }, {  }, {  }, {  }, {  }, {  } ] }
+{ "id": 13, "t13": [ { "$13": [ { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] }, { "$13": [ { "state": "IL", "country": "US", "zip_code": 92863 }, { "state": "OR", "country": "US", "zip_code": null } ] } ] }
+{ "id": 14, "t14": [ { "$14": [ { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] }, { "$14": [ { "state": "IL", "country": "US", "zip_code": 92863 }, { "state": "OR", "country": "US", "zip_code": null } ] } ] }
+{ "id": 15, "t15": [ { "$15": null }, { "$15": null } ] }
+{ "id": 16, "t16": [ {  }, {  } ] }
+{ "id": 17, "t17": [ { "$17": null } ] }
+{ "id": 18, "t18": [ { "$18": null } ] }
+{ "id": 19, "t19": [ { "$19": null } ] }
+{ "id": 20, "t20": [ { "$20": null } ] }
+{ "id": 21, "t21": [ {  } ] }
+{ "id": 22, "t22": [ {  } ] }
+{ "id": 23, "t23": [ {  } ] }
+{ "id": 24, "t24": [ { "$24": [ 5, 2, 3, 4, 1 ] } ] }
+{ "id": 25, "t25": [ { "$25": [ 1, 4, 3, 2, 5 ] } ] }
+{ "id": 26, "t26": [ { "$26": null } ] }
\ No newline at end of file
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 a2a5f0a..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/join/nlj-outer/nlj-outer.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.10.adm
new file mode 100644
index 0000000..f768f58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.10.adm
@@ -0,0 +1 @@
+1535
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.11.adm
new file mode 100644
index 0000000..f768f58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.11.adm
@@ -0,0 +1 @@
+1535
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.20.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.20.adm
new file mode 100644
index 0000000..6bf42b9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.20.adm
@@ -0,0 +1 @@
+757
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.21.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.21.adm
new file mode 100644
index 0000000..6bf42b9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.21.adm
@@ -0,0 +1 @@
+757
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.30.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.30.adm
new file mode 100644
index 0000000..6bf42b9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.30.adm
@@ -0,0 +1 @@
+757
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.31.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.31.adm
new file mode 100644
index 0000000..6bf42b9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.31.adm
@@ -0,0 +1 @@
+757
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.40.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.40.adm
new file mode 100644
index 0000000..6822652
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.40.adm
@@ -0,0 +1 @@
+793
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.41.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.41.adm
new file mode 100644
index 0000000..6822652
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.41.adm
@@ -0,0 +1 @@
+793
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.50.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.50.adm
new file mode 100644
index 0000000..000b173
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.50.adm
@@ -0,0 +1 @@
+1485
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.51.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.51.adm
new file mode 100644
index 0000000..000b173
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/nlj-outer/nlj-outer.51.adm
@@ -0,0 +1 @@
+1485
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/offset_without_limit/offset_without_limit.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/offset_without_limit/offset_without_limit.7.adm
new file mode 100644
index 0000000..d29535d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/offset_without_limit/offset_without_limit.7.adm
@@ -0,0 +1 @@
+{ "x": 2, "count": 1 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/active_requests/active_requests.4.regex b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/active_requests/active_requests.4.regex
index e2d166b..695472c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/active_requests/active_requests.4.regex
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/active_requests/active_requests.4.regex
@@ -1 +1 @@
-/memory\D+229376/
\ No newline at end of file
+/memory\D+0/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/completed_requests/completed_requests.3.regex b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/completed_requests/completed_requests.3.regex
index e2d166b..695472c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/completed_requests/completed_requests.3.regex
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/completed_requests/completed_requests.3.regex
@@ -1 +1 @@
-/memory\D+229376/
\ No newline at end of file
+/memory\D+0/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.3.adm
new file mode 100644
index 0000000..190adc2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-3316/query-ASTERIXDB-3316.3.adm
@@ -0,0 +1 @@
+{ "sub_query1": [ { "u": "col1" } ], "sub_query2": [ { "u": "col1" } ], "x_id": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm
index 800859b..94bb2b5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm
@@ -1 +1 @@
-{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": 1 }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136, "field": "value" } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)", "field": { "a": 1 } } ] }
+{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": 1 }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "field": "value", "name": "Chang Ewing", "followers_count": 32136 } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)", "field": { "a": 1 } } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.7.adm
new file mode 100644
index 0000000..3bbbabb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.7.adm
@@ -0,0 +1 @@
+{ "t1c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ], "name": "John" }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "name": "John" }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "name": "John" } ], "t1o": [ { "_id": 4, "name": "John", "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, { "_id": 5, "name": "John", "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "name": "John", "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t2c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ], "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } } ], "t2o": [ { "_id": 4, "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, { "_id": 5, "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t3c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ], "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } } ], "t3o": [ { "_id": 4, "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] }, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, { "_id": 5, "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] }, "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] }, "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t4c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ], "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } ] }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } ] }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } ] } ], "t4o": [ { "_id": 4, "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } ], "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, { "_id": 5, "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } ], "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } ], "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t5c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ], "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } ] }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } ] }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } ] } ], "t5o": [ { "_id": 4, "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } ], "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, { "_id": 5, "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } ], "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } ], "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.5.adm
new file mode 100644
index 0000000..e20e752
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.5.adm
@@ -0,0 +1 @@
+{ "res": { "id": 2, "f": 4 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.6.adm
new file mode 100644
index 0000000..80023c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.6.adm
@@ -0,0 +1 @@
+{ "id": 1, "b1": 3, "dup": 5, "a1": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm
index 4b97741..2947405 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm
@@ -1 +1 @@
-{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": "replaced" }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136, "field": "value" } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)", "field": { "a": 1 } } ] }
+{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": "replaced" }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "field": "value", "name": "Chang Ewing", "followers_count": 32136 } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)", "field": { "a": 1 } } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.7.adm
new file mode 100644
index 0000000..32b297d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.7.adm
@@ -0,0 +1 @@
+{ "t1c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ], "name": "John" }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "name": "John" }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "name": "John" } ], "t1o": [ { "_id": 4, "name": "John", "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, { "_id": 5, "name": "John", "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "name": "John", "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t2c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ], "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } } ], "t2o": [ { "_id": 4, "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, { "_id": 5, "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "newFriend": { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t3c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ], "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } } ], "t3o": [ { "_id": 4, "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] }, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, { "_id": 5, "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] }, "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "newFriend": { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] }, "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t4c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ], "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } ] }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } ] }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } ] } ], "t4o": [ { "_id": 4, "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } ], "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, { "_id": 5, "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } ], "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "newFriends": [ { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] } ], "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t5c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ], "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } ] }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } ] }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } ] } ], "t5o": [ { "_id": 4, "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } ], "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, { "_id": 5, "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } ], "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_id": 4 } ] }, { "_id": 6, "newFriends": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ] } ], "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [ { "_id": 4 }, { "_id": 5 } ] } ], "t6c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [ { "_id": 8 }, { "_id": 9 } ] }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 8 }, { "_id": 9 } ] }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 8 }, { "_id": 9 } ] } ] }
\ 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..03900f3 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",
-                            "time": "R{[0-9.]+}",
-                            "disk-io": "R{[0-9.]+}"
-                        },
-                        {
-                            "name": "Index Search",
-                            "time": "R{[0-9.]+}",
-                            "disk-io": "R{[0-9.]+}"
+                            "name": "R{.+}",
+                            "run-time": "R{[0-9.]+}",
+                            "runtime-id": "R{.+}"
                         },
                         {
                             "name": "R{.+}",
-                            "time": "R{[0-9.]+}",
-                            "disk-io": "R{[0-9.]+}"
+                            "run-time": "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{.+}",
+                            "run-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,39 @@
                   "partition-send-profile": [],
                   "counters": [
                     {
-                      "name": "R{.+}",
-                      "time": "R{[0-9.]+}",
-                      "disk-io": "R{[0-9.]+}"
+                        "name": "R{.+}",
+                        "run-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{.+}",
+                        "run-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": "R{.+}",
+                        "run-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": "R{.+}",
+                        "run-time": "R{[0-9.]+}",
+                        "runtime-id": "R{.+}"
                     }
                   ]
                 }
             ]
-        }
-    ]
+    }]
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/non-unary-subplan/non-unary-subplan.3.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/non-unary-subplan/non-unary-subplan.3.regexjson
new file mode 100644
index 0000000..e59f095
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/non-unary-subplan/non-unary-subplan.3.regexjson
@@ -0,0 +1,241 @@
+{
+    "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": [
+        {
+            "node-id": "R{.+}",
+            "counters": [],
+            "tasks": [
+                {
+                    "activity-id": "R{[A-Z0-9.:]+}",
+                    "partition": "R{[0-9]+}",
+                    "attempt": "R{[0-9]+}",
+                    "partition-send-profile": [],
+                    "counters": [
+                        {
+                            "name": "R{.+}",
+                            "run-time": "R{[0-9.]+}",
+                            "runtime-id": "R{.+}"
+                        }
+                    ]
+                },
+                {
+                    "activity-id": "R{[A-Z0-9.:]+}",
+                    "partition": "R{[0-9]+}",
+                    "attempt": "R{[0-9]+}",
+                    "partition-send-profile": [],
+                    "counters": [
+                        {
+                            "name": "R{ANID:ODID:[0-9]:0\\.1 - MicroOp Subplan(?:.|\n)+}",
+                            "run-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": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-time": "R{[0-9.]+}",
+                            "runtime-id": "R{.+}"
+                        }
+                    ]
+                },
+                {
+                    "activity-id": "R{[A-Z0-9.:]+}",
+                    "partition": "R{[0-9]+}",
+                    "attempt": "R{[0-9]+}",
+                    "partition-send-profile": [
+                        {
+                            "partition-id": {
+                                "job-id": "R{[A-Z0-9.:]+}",
+                                "connector-id": "R{[A-Z0-9.:]+}",
+                                "sender-index": "R{[0-9]+}",
+                                "receiver-index": "R{[0-9]+}"
+                            },
+                            "open-time": "R{[0-9]+}",
+                            "close-time": "R{[0-9]+}",
+                            "offset": "R{[0-9]+}",
+                            "frame-times": [
+                                0
+                            ],
+                            "resolution": 1
+                        }
+                    ],
+                    "counters": [
+                        {
+                            "name": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-time": "R{[0-9.]+}",
+                            "runtime-id": "R{.+}"
+                        },
+                        {
+                            "name": "R{.+}",
+                            "run-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.]+}"
+                        }
+                    ]
+                }
+            ]
+    }]
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.3.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.3.regexjson
new file mode 100644
index 0000000..e6d1c0a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.3.regexjson
@@ -0,0 +1,75 @@
+{
+    "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": [
+        {
+            "node-id": "R{.+}",
+            "counters": [],
+            "tasks": [
+                {
+                    "activity-id": "R{[A-Z0-9.:]+}",
+                    "partition": "R{[0-9]+}",
+                    "attempt": "R{[0-9]+}",
+                    "partition-send-profile": [],
+                    "counters": [
+                        {
+                            "name": "R{.+}",
+                            "run-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": "R{.+}",
+                            "run-time": "R{[0-9.]+}",
+                            "runtime-id": "R{.+}"
+                        },
+                        {
+                            "name": "R{.+}",
+                            "run-time": "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{.+}",
+                            "run-time": "R{5.+}",
+                            "runtime-id": "R{.+}",
+                            "cardinality-out": 10,
+                            "avg-tuple-size": 38,
+                            "min-tuple-size": 38,
+                            "max-tuple-size": 38
+                        },
+                        {
+                            "name": "R{.+}",
+                            "run-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": "R{.+}",
+                          "run-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.]+}"
+                        }
+                    ]
+                }
+            ]
+    }]
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.4.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.4.regexjson
new file mode 100644
index 0000000..e3e7647
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.4.regexjson
@@ -0,0 +1,166 @@
+{
+    "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": [
+        {
+            "node-id": "R{.+}",
+            "counters": [],
+            "tasks": [
+                {
+                    "activity-id": "R{[A-Z0-9.:]+}",
+                    "partition": "R{[0-9]+}",
+                    "attempt": "R{[0-9]+}",
+                    "partition-send-profile": [
+                        {
+                            "partition-id": {
+                                "job-id": "R{[A-Z0-9.:]+}",
+                                "connector-id": "R{[A-Z0-9.:]+}",
+                                "sender-index": "R{[0-9]+}",
+                                "receiver-index": "R{[0-9]+}"
+                            },
+                            "open-time": "R{[0-9]+}",
+                            "close-time": "R{[0-9]+}",
+                            "offset": "R{[0-9]+}",
+                            "frame-times": [
+                                0
+                            ],
+                            "resolution": 1
+                        }
+                    ],
+                    "counters": [
+                        {
+                            "name": "R{.+}",
+                            "run-time": "R{[0-9.]+}",
+                            "runtime-id": "R{.+}"
+                        }
+                    ]
+                },
+                {
+                  "activity-id": "R{[A-Z0-9.:]+}",
+                  "partition": "R{[0-9]+}",
+                  "attempt": "R{[0-9]+}",
+                  "partition-send-profile": [],
+                  "counters": [
+                    {
+                        "name": "R{.+}",
+                        "run-time": "R{[0-9.]+}",
+                        "runtime-id": "R{.+}"
+                    }
+                  ]
+                },
+                {
+                  "activity-id": "R{[A-Z0-9.:]+}",
+                  "partition": "R{[0-9]+}",
+                  "attempt": "R{[0-9]+}",
+                  "partition-send-profile": [],
+                  "counters": [
+                    {
+                        "name": "R{.+}",
+                        "run-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": "R{.+}",
+                        "run-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": "R{.+}",
+                        "run-time": "R{[0-9.]+}",
+                        "runtime-id": "R{.+}"
+                    },
+                    {
+                        "name": "R{.+}",
+                        "run-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.]+}"
+                    }
+                  ]
+                },
+                {
+                  "activity-id": "R{[A-Z0-9.:]+}",
+                  "partition": "R{[0-9]+}",
+                  "attempt": "R{[0-9]+}",
+                  "partition-send-profile": [],
+                  "counters": [
+                    {
+                        "name": "R{.+}",
+                        "run-time": "R{[0-9.]+}",
+                        "runtime-id": "R{.+}",
+                        "cardinality-out": 10,
+                        "avg-tuple-size": 140,
+                        "min-tuple-size": 137,
+                        "max-tuple-size": 151
+                    },
+                    {
+                        "name": "R{.+}",
+                        "run-time": "R{[0-9.]+}",
+                        "runtime-id": "R{.+}",
+                        "cardinality-out": 5,
+                        "avg-tuple-size": 145,
+                        "min-tuple-size": 142,
+                        "max-tuple-size": 151
+                    },
+                    {
+                        "name": "R{.+}",
+                        "run-time": "R{[0-9.]+}",
+                        "runtime-id": "R{.+}"
+                    },
+                    {
+                        "name": "R{.+}",
+                        "run-time": "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{.+}",
+                        "run-time": "R{[0-9.]+}",
+                        "runtime-id": "R{.+}",
+                        "cardinality-out": 5,
+                        "avg-tuple-size": 16,
+                        "min-tuple-size": 14,
+                        "max-tuple-size": 19
+                    },
+                    {
+                        "name": "R{.+}",
+                        "run-time": "R{5.+}",
+                        "runtime-id": "R{.+}",
+                        "cardinality-out": 5,
+                        "avg-tuple-size": 161,
+                        "min-tuple-size": 156,
+                        "max-tuple-size": 170
+                    },
+                    {
+                        "name": "R{.+}",
+                        "run-time": "R{[0-9.]+}",
+                        "runtime-id": "R{.+}",
+                        "cardinality-out": 1,
+                        "avg-tuple-size": 0,
+                        "min-tuple-size": 0,
+                        "max-tuple-size": 0
+                    }
+                  ]
+                }
+            ]
+    }]
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.5.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.5.regexjson
new file mode 100644
index 0000000..98d7930
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/profile/sleep/sleep.5.regexjson
@@ -0,0 +1,235 @@
+{
+    "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": [
+        {
+            "node-id": "R{.+}",
+            "counters": [],
+            "tasks": [
+                {
+                    "activity-id": "R{[A-Z0-9.:]+}",
+                    "partition": "R{[0-9]+}",
+                    "attempt": "R{[0-9]+}",
+                    "partition-send-profile": [
+                        {
+                            "partition-id": {
+                                "job-id": "R{[A-Z0-9.:]+}",
+                                "connector-id": "R{[A-Z0-9.:]+}",
+                                "sender-index": "R{[0-9]+}",
+                                "receiver-index": "R{[0-9]+}"
+                            },
+                            "open-time": "R{[0-9]+}",
+                            "close-time": "R{[0-9]+}",
+                            "offset": "R{[0-9]+}",
+                            "frame-times": [
+                                0
+                            ],
+                            "resolution": 1
+                        }
+                    ],
+                    "counters": [
+                        {
+                            "name": "R{.+}",
+                            "run-time": "R{[0-9.]+}",
+                            "runtime-id": "R{.+}"
+                        }
+                    ]
+                },
+                {
+                  "activity-id": "R{[A-Z0-9.:]+}",
+                  "partition": "R{[0-9]+}",
+                  "attempt": "R{[0-9]+}",
+                  "partition-send-profile": [],
+                  "counters": [
+                    {
+                        "name": "R{.+}",
+                        "run-time": "R{[0-9.]+}",
+                        "runtime-id": "R{.+}"
+                    }
+                  ]
+                },
+                {
+                  "activity-id": "R{[A-Z0-9.:]+}",
+                  "partition": "R{[0-9]+}",
+                  "attempt": "R{[0-9]+}",
+                  "partition-send-profile": [],
+                  "counters": [
+                    {
+                        "name": "R{.+}",
+                        "run-time": "R{5.+}",
+                        "runtime-id": "R{.+}",
+                        "cardinality-out": 3,
+                        "avg-tuple-size": 75,
+                        "min-tuple-size": 67,
+                        "max-tuple-size": 82
+                    },
+                    {
+                        "name": "R{.+}",
+                        "run-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": "R{.+}",
+                        "run-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": "R{.+}",
+                        "run-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": "R{.+}",
+                        "run-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": "R{.+}",
+                        "run-time": "R{[0-9.]+}",
+                        "runtime-id": "R{.+}"
+                    }
+                  ]
+                },
+                {
+                    "activity-id": "R{[A-Z0-9.:]+}",
+                    "partition": "R{[0-9]+}",
+                    "attempt": "R{[0-9]+}",
+                    "partition-send-profile": [
+                        {
+                            "partition-id": {
+                                "job-id": "R{[A-Z0-9.:]+}",
+                                "connector-id": "R{[A-Z0-9.:]+}",
+                                "sender-index": "R{[0-9]+}",
+                                "receiver-index": "R{[0-9]+}"
+                            },
+                            "open-time": "R{[0-9]+}",
+                            "close-time": "R{[0-9]+}",
+                            "offset": "R{[0-9]+}",
+                            "frame-times": [
+                                0
+                            ],
+                            "resolution": 1
+                        }
+                    ],
+                    "counters": [
+                        {
+                            "name": "R{.+}",
+                            "run-time": "R{[0-9.]+}",
+                            "runtime-id": "R{.+}"
+                        }
+                    ]
+                },
+                {
+                  "activity-id": "R{[A-Z0-9.:]+}",
+                  "partition": "R{[0-9]+}",
+                  "attempt": "R{[0-9]+}",
+                  "partition-send-profile": [],
+                  "counters": [
+                    {
+                        "name": "R{.+}",
+                        "run-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": "R{.+}",
+                        "run-time": "R{[0-9.]+}",
+                        "runtime-id": "R{.+}"
+                    }
+                  ]
+                },
+                {
+                  "activity-id": "R{[A-Z0-9.:]+}",
+                  "partition": "R{[0-9]+}",
+                  "attempt": "R{[0-9]+}",
+                  "partition-send-profile": [],
+                  "counters": [
+                    {
+                        "name": "R{.+}",
+                        "run-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": "R{.+}",
+                        "run-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": "R{.+}",
+                        "run-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": "R{.+}",
+                        "run-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": "R{.+}",
+                        "run-time": "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{.+}",
+                        "run-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": "R{.+}",
+                        "run-time": "R{[0-9.]+}",
+                        "runtime-id": "R{.+}"
+                    }
+                  ]
+                }
+            ]
+    }]
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.plan
index 6faed2f..c69e8a0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.plan
@@ -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.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.plan
index 6faed2f..c69e8a0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.plan
@@ -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/union/union_type_cast/union_type_cast.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_type_cast/union_type_cast.3.adm
new file mode 100644
index 0000000..cefb946
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_type_cast/union_type_cast.3.adm
@@ -0,0 +1,40 @@
+{ "ds": "message1", "id": 1, "no_in_response_to": { "message-id": 1, "author-id": 3, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)" } }
+{ "ds": "message2", "id": 1, "no_author_id": { "message-id": 1, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)" } }
+{ "ds": "user", "id": 1, "user": { "id": 1, "alias": "Margarita", "name": "MargaritaStoddard", "user-since": datetime("2012-08-20T10:10:00.000"), "friend-ids": {{ 2, 3, 6, 10 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2006-08-06") } ] } }
+{ "ds": "message1", "id": 2, "no_in_response_to": { "message-id": 2, "author-id": 1, "sender-location": point("41.66,80.87"), "message": " dislike iphone its touch-screen is horrible" } }
+{ "ds": "message2", "id": 2, "no_author_id": { "message-id": 2, "in-response-to": 4, "sender-location": point("41.66,80.87"), "message": " dislike iphone its touch-screen is horrible" } }
+{ "ds": "user", "id": 2, "user": { "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27") } ] } }
+{ "ds": "message1", "id": 3, "no_in_response_to": { "message-id": 3, "author-id": 2, "sender-location": point("48.09,81.01"), "message": " like samsung the plan is amazing" } }
+{ "ds": "message2", "id": 3, "no_author_id": { "message-id": 3, "in-response-to": 4, "sender-location": point("48.09,81.01"), "message": " like samsung the plan is amazing" } }
+{ "ds": "user", "id": 3, "user": { "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] } }
+{ "ds": "message1", "id": 4, "no_in_response_to": { "message-id": 4, "author-id": 1, "sender-location": point("37.73,97.04"), "message": " can't stand at&t the network is horrible:(" } }
+{ "ds": "message2", "id": 4, "no_author_id": { "message-id": 4, "in-response-to": 2, "sender-location": point("37.73,97.04"), "message": " can't stand at&t the network is horrible:(" } }
+{ "ds": "user", "id": 4, "user": { "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08") } ] } }
+{ "ds": "message1", "id": 5, "no_in_response_to": { "message-id": 5, "author-id": 6, "sender-location": point("34.7,90.76"), "message": " love sprint the customization is mind-blowing" } }
+{ "ds": "message2", "id": 5, "no_author_id": { "message-id": 5, "in-response-to": 2, "sender-location": point("34.7,90.76"), "message": " love sprint the customization is mind-blowing" } }
+{ "ds": "user", "id": 5, "user": { "id": 5, "alias": "Von", "name": "VonKemble", "user-since": datetime("2010-01-05T10:10:00.000"), "friend-ids": {{ 3, 6, 10 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2010-11-27") } ] } }
+{ "ds": "message1", "id": 6, "no_in_response_to": { "message-id": 6, "author-id": 2, "sender-location": point("31.5,75.56"), "message": " like t-mobile its platform is mind-blowing" } }
+{ "ds": "message2", "id": 6, "no_author_id": { "message-id": 6, "in-response-to": 1, "sender-location": point("31.5,75.56"), "message": " like t-mobile its platform is mind-blowing" } }
+{ "ds": "user", "id": 6, "user": { "id": 6, "alias": "Willis", "name": "WillisWynne", "user-since": datetime("2005-01-17T10:10:00.000"), "friend-ids": {{ 1, 3, 7 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2009-05-15") } ] } }
+{ "ds": "message1", "id": 7, "no_in_response_to": { "message-id": 7, "author-id": 5, "sender-location": point("32.91,85.05"), "message": " dislike sprint the speed is horrible" } }
+{ "ds": "message2", "id": 7, "no_author_id": { "message-id": 7, "in-response-to": 15, "sender-location": point("32.91,85.05"), "message": " dislike sprint the speed is horrible" } }
+{ "ds": "user", "id": 7, "user": { "id": 7, "alias": "Suzanna", "name": "SuzannaTillson", "user-since": datetime("2012-08-07T10:10:00.000"), "friend-ids": {{ 6 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2011-04-19") } ] } }
+{ "ds": "message1", "id": 8, "no_in_response_to": { "message-id": 8, "author-id": 1, "sender-location": point("40.33,80.87"), "message": " like verizon the 3G is awesome:)" } }
+{ "ds": "message2", "id": 8, "no_author_id": { "message-id": 8, "in-response-to": 11, "sender-location": point("40.33,80.87"), "message": " like verizon the 3G is awesome:)" } }
+{ "ds": "user", "id": 8, "user": { "id": 8, "alias": "Nila", "name": "NilaMilliron", "user-since": datetime("2008-01-01T10:10:00.000"), "friend-ids": {{ 3 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2010-02-28") } ] } }
+{ "ds": "message1", "id": 9, "no_in_response_to": { "message-id": 9, "author-id": 3, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good" } }
+{ "ds": "message2", "id": 9, "no_author_id": { "message-id": 9, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good" } }
+{ "ds": "user", "id": 9, "user": { "id": 9, "alias": "Woodrow", "name": "WoodrowNehling", "user-since": datetime("2005-09-20T10:10:00.000"), "friend-ids": {{ 3, 10 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2003-04-22"), "end-date": date("2009-12-13") } ] } }
+{ "ds": "message1", "id": 10, "no_in_response_to": { "message-id": 10, "author-id": 1, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible" } }
+{ "ds": "message2", "id": 10, "no_author_id": { "message-id": 10, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible" } }
+{ "ds": "user", "id": 10, "user": { "id": 10, "alias": "Bram", "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000"), "friend-ids": {{ 1, 5, 9 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2007-06-05"), "end-date": date("2011-11-05") } ] } }
+{ "ds": "message1", "id": 11, "no_in_response_to": { "message-id": 11, "author-id": 1, "sender-location": point("38.97,77.49"), "message": " can't stand at&t its plan is terrible" } }
+{ "ds": "message2", "id": 11, "no_author_id": { "message-id": 11, "in-response-to": 1, "sender-location": point("38.97,77.49"), "message": " can't stand at&t its plan is terrible" } }
+{ "ds": "message1", "id": 12, "no_in_response_to": { "message-id": 12, "author-id": 10, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(" } }
+{ "ds": "message2", "id": 12, "no_author_id": { "message-id": 12, "in-response-to": 6, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(" } }
+{ "ds": "message1", "id": 13, "no_in_response_to": { "message-id": 13, "author-id": 10, "sender-location": point("42.77,78.92"), "message": " dislike iphone the voice-command is bad:(" } }
+{ "ds": "message2", "id": 13, "no_author_id": { "message-id": 13, "in-response-to": 4, "sender-location": point("42.77,78.92"), "message": " dislike iphone the voice-command is bad:(" } }
+{ "ds": "message1", "id": 14, "no_in_response_to": { "message-id": 14, "author-id": 9, "sender-location": point("41.33,85.28"), "message": " love at&t its 3G is good:)" } }
+{ "ds": "message2", "id": 14, "no_author_id": { "message-id": 14, "in-response-to": 12, "sender-location": point("41.33,85.28"), "message": " love at&t its 3G is good:)" } }
+{ "ds": "message1", "id": 15, "no_in_response_to": { "message-id": 15, "author-id": 7, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome" } }
+{ "ds": "message2", "id": 15, "no_author_id": { "message-id": 15, "in-response-to": 11, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome" } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_type_cast/union_type_cast.4.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_type_cast/union_type_cast.4.plan
new file mode 100644
index 0000000..d0aa765
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_type_cast/union_type_cast.4.plan
@@ -0,0 +1,72 @@
+distribute result [$#1] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    project ([$#1]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- STREAM_PROJECT  |PARTITIONED|
+      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- SORT_MERGE_EXCHANGE [$$102(ASC) ]  |PARTITIONED|
+        order (ASC, $$102) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+        -- STABLE_SORT [$$102(ASC)]  |PARTITIONED|
+          exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            union ($$64, $$140, $#1) ($$103, $$70, $$102) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+            -- UNION_ALL  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                union ($$139, $$141, $$64) ($$68, $$69, $$103) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- UNION_ALL  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    project ([$$139, $$68]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      assign [$$139] <- [cast({"id": $$68, "no_in_response_to": object-remove($$s, "in-response-to")})] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- ASSIGN  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- REPLICATE  |PARTITIONED|
+                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              data-scan []<-[$$68, $$s] <- TinySocial.FacebookMessages [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    project ([$$141, $$69]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      assign [$$141] <- [cast({"id": $$69, "user": $$t})] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- ASSIGN  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$69, $$t] <- TinySocial.FacebookUsers [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- DATASOURCE_SCAN  |PARTITIONED|
+                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$140, $$70]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- STREAM_PROJECT  |PARTITIONED|
+                  assign [$$140] <- [cast({"id": $$70, "no_author_id": object-remove($$s, "author-id")})] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ASSIGN  |PARTITIONED|
+                    project ([$$70, $$s]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      assign [$$70, $$s] <- [$$68, $$s] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                      -- ASSIGN  |PARTITIONED|
+                        exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          replicate [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                          -- REPLICATE  |PARTITIONED|
+                            exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              data-scan []<-[$$68, $$s] <- TinySocial.FacebookMessages [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                                  -- EMPTY_TUPLE_SOURCE  |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..bd85d0e 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,8 @@
 \s*\Q"metrics": {\E
 \s*\Q"elapsedTime": "\E[^"]+\Q",\E
 \s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
+\s*\Q"queueWaitTime": "\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..dce84fe 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,8 @@
 \s*\Q"metrics": {\E
 \s*\Q"elapsedTime": "\E[^"]+\Q",\E
 \s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
+\s*\Q"queueWaitTime": "\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..6b1931a 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,8 @@
 \s*\Q"metrics": {\E
 \s*\Q"elapsedTime": "\E[^"]+\Q",\E
 \s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
+\s*\Q"queueWaitTime": "\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..39fdb09 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,9 +12,12 @@
 \s*\Q"metrics": {\E
 \s*\Q"elapsedTime": "\E[^"]+\Q",\E
 \s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
+\s*\Q"queueWaitTime": "\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
+\s*\Q"bufferCacheHitRatio": "\E[^"]+\Q",\E
 \s*\Q"warningCount": 2\E
 \s*\Q}\E
 \s*\Q}\E\s*
\ No newline at end of file
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..6b1931a 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,8 @@
 \s*\Q"metrics": {\E
 \s*\Q"elapsedTime": "\E[^"]+\Q",\E
 \s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
+\s*\Q"queueWaitTime": "\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..bd85d0e 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,8 @@
 \s*\Q"metrics": {\E
 \s*\Q"elapsedTime": "\E[^"]+\Q",\E
 \s*\Q"executionTime": "\E[^"]+\Q",\E
+\s*\Q"compileTime": "\E[^"]+\Q",\E
+\s*\Q"queueWaitTime": "\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/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 a9fbbe3..e06a899 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -101,6 +101,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="api">
+      <compilation-unit name="cache-residency">
+        <output-dir compare="Text">cache-residency</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>
@@ -3111,6 +3116,21 @@
         <expected-error>Input contains different list types (in line 25, at column 8)</expected-error>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_binary_search">
+        <output-dir compare="Text">array_binary_search</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_move">
+        <output-dir compare="Text">array_move</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_swap">
+        <output-dir compare="Text">array_swap</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="bitwise">
     <test-case FilePath="bitwise">
@@ -6680,6 +6700,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="join">
+      <compilation-unit name="nlj-outer">
+        <output-dir compare="Text">nlj-outer</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>
@@ -7286,6 +7311,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-3316">
+        <output-dir compare="Text">query-ASTERIXDB-3316</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
       <compilation-unit name="query-ASTERIXDB-3334">
         <output-dir compare="Text">query-ASTERIXDB-3334</output-dir>
       </compilation-unit>
@@ -9979,6 +10009,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">
@@ -15074,6 +15123,11 @@
         <output-dir compare="Text">query-ASTERIXDB-1354</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="union">
+      <compilation-unit name="union_type_cast">
+        <output-dir compare="Text">union_type_cast</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="upsert">
     <test-case FilePath="upsert">
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/config/CompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
index 325c238..9296339 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
@@ -105,6 +105,10 @@
                 BOOLEAN,
                 AlgebricksConfig.ARRAY_INDEX_DEFAULT,
                 "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(
@@ -184,6 +188,8 @@
 
     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();
@@ -266,6 +272,10 @@
         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);
     }
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 ae70475..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,8 @@
         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,
@@ -103,6 +105,7 @@
         physOptConf.setMinMemoryAllocation(minMemoryAllocation);
         physOptConf.setArrayIndexEnabled(arrayIndex);
         physOptConf.setExternalScanBufferSize(externalScanBufferSize);
+        physOptConf.setBatchLookup(batchLookup);
         physOptConf.setCBOMode(cbo);
         physOptConf.setCBOTestMode(cboTest);
         physOptConf.setForceJoinOrderMode(forceJoinOrder);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
index 4fc9dd6..507a227 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
@@ -150,7 +150,7 @@
         IndexInfo iInfo = dsr == null ? null : dsr.getIndexInfo(resourceID);
 
         if (dsr == null || iInfo == null) {
-            throw HyracksDataException.create(ErrorCode.INDEX_DOES_NOT_EXIST);
+            throw HyracksDataException.create(ErrorCode.INDEX_DOES_NOT_EXIST, resourcePath);
         }
 
         PrimaryIndexOperationTracker opTracker = dsr.getOpTracker(iInfo.getPartition());
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/GlobalVirtualBufferCache.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/GlobalVirtualBufferCache.java
index 0bb9bd5..a7210bc 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/GlobalVirtualBufferCache.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/GlobalVirtualBufferCache.java
@@ -273,6 +273,11 @@
         return page;
     }
 
+    @Override
+    public ICachedPage pin(long dpid, boolean newPage, boolean incrementStats) throws HyracksDataException {
+        return pin(dpid, newPage);
+    }
+
     private void incrementFilteredMemoryComponentUsage(long dpid, int pages) {
         if (filteredMemoryComponentMaxNumPages > 0) {
             // update memory usage of filtered index
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java
index f56e5c0..a098a29 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java
@@ -56,7 +56,6 @@
 import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
 import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
 import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
-import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -175,11 +174,11 @@
         synchronized (lsmIndex.getOperationTracker()) {
             List<ILSMDiskComponent> diskComponents = lsmIndex.getDiskComponents();
             if (diskComponents.isEmpty()) {
-                LOGGER.log(Level.INFO, "There are no disk components");
+                LOGGER.info("there are no disk components for {}", lsmIndex);
                 return LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID;
             }
             if (deletedComponents.contains(diskComponents.get(diskComponents.size() - 1))) {
-                LOGGER.log(Level.INFO, "All disk components have been deleted");
+                LOGGER.info("all disk components have been deleted for {}", lsmIndex);
                 return LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID;
             }
             int mostRecentComponentIndex = 0;
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 074245c..4835073 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -282,7 +282,7 @@
 3006 = Illegal field %1$s in closed type %2$s
 3007 = Twitter4J library not found!
 3008 = Unable to ingest data
-3009 = Exception in get record type %1$s for feed
+3009 = Exception in get record type %1$s for ingestion
 3010 = Does not support Hive data with list of non-primitive types
 3011 = Cannot get hive type for field of type %1$s
 3012 = Failed to get columns of record
@@ -379,7 +379,7 @@
 3105 = %1$s is already registered
 3106 = %1$s is not registered
 3107 = Active Notification Handler is already suspended
-3110 = Feed failed while reading a new record
+3110 = Ingestion failed while reading a new record
 3111 = Feed %1$s is not connected to any dataset
 3112 = Array/Multiset item cannot be null
 3113 = Failed to parse record
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-doc/src/main/grammar/sqlpp.ebnf b/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
index b237f30..af67e33 100644
--- a/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
+++ b/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
@@ -59,7 +59,7 @@
 
 StreamGenerator::= FromClause LetClause? WhereClause? (GroupByClause LetClause? HavingClause?)?
 
-SelectClause ::= "SELECT" ("DISTINCT" | "ALL")? ( "VALUE" Expr | Projection ("," Projection)*)
+SelectClause ::= "SELECT" ("DISTINCT" | "ALL")? ( "VALUE" Expr | Projection ("," Projection)* ( "EXCLUDE" Identifier (("," | ".") Identifier)* )?  )
 
 Projection ::= (Expr ("AS"? Identifier)?) | (VariableRef "." "*") | "*"
 
@@ -162,7 +162,7 @@
 
 CreateType ::= "CREATE" "TYPE" QualifiedName ("IF" "NOT" "EXISTS")? "AS" ObjectTypeDef
 
-ObjectTypeDef ::= ("CLOSED" | "OPEN")? "{" ObjectField ("," ObjectField)* "}"
+ObjectTypeDef ::= ("CLOSED" | "OPEN")? "{" ( ObjectField ("," ObjectField)* )? "}"
 
 ObjectField ::= Identifier ":" Identifier "?"?
 
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md
index 04a65d1..b8dc3bf 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md
@@ -248,7 +248,7 @@
 
 ##### Example
 
-(Q3.5) Returns all of the different cities in the `customers` dataset.
+(Q3.5a) Returns all of the different cities in the `customers` dataset.
 
     FROM customers AS c
     SELECT DISTINCT c.address.city;
@@ -270,6 +270,32 @@
         }
     ]
 
+### <a id="Select_exclude">SELECT EXCLUDE</a>
+The `EXCLUDE` keyword is used to remove one or more fields that would otherwise be returned from the `SELECT` clause.
+Conceptually, the scope of the `EXCLUDE` clause is the output of the `SELECT` clause itself.
+In a Stream Generator with both `DISTINCT` and `EXCLUDE` clauses, the `DISTINCT` clause is applied after the `EXCLUDE` clause.
+
+##### Example
+
+(Q3.5b) For the customer with `custid = C13`, return their information _excluding_ the `zipcode` field inside the `address` object and the top-level `name` field.
+
+    FROM customers AS c
+    WHERE c.custid = "C13"
+    SELECT c.* EXCLUDE address.zipcode, name;
+
+Result:
+
+    [
+        {
+            "custid": "C13",
+            "address": {
+                "street": "201 Main St.",
+                "city": "St. Louis, MO"
+            },
+            "rating": 750
+        }
+    ]
+
 ### <a id="Unnamed_projections">Unnamed Projections</a>
 
 Similar to standard SQL, the query language supports unnamed projections (a.k.a, unnamed `SELECT` clause items), for which names are generated rather than user-provided.
diff --git a/asterixdb/asterix-docker/pom.xml b/asterixdb/asterix-docker/pom.xml
deleted file mode 100644
index 39bd8c7..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.2-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 ee4ac89..4306864 100644
--- a/asterixdb/asterix-external-data/pom.xml
+++ b/asterixdb/asterix-external-data/pom.xml
@@ -212,7 +212,11 @@
               <usedDependency>io.netty:netty-codec-http2</usedDependency>
               <usedDependency>io.netty:netty-transport-native-unix-common</usedDependency>
               <usedDependency>io.netty:netty-handler-proxy</usedDependency>
+              <usedDependency>io.netty:netty-handler-proxy</usedDependency>
             </usedDependencies>
+            <ignoredUnusedDeclaredDependencies>
+              <unusedDeclaredDependency>io.grpc:grpc-census:*</unusedDeclaredDependency>
+            </ignoredUnusedDeclaredDependencies>
           </configuration>
         </plugin>
       </plugins>
@@ -514,6 +518,15 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>com.google.cloud.bigdataoss</groupId>
+      <artifactId>gcs-connector</artifactId>
+    </dependency>
+    <!-- explicitly include grpc-census, to ensure the correct version -->
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-census</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/dataflow/FeedRecordDataFlowController.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
index 4279ebd..efa93ab 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
@@ -98,7 +98,7 @@
                 }
             }
         } catch (HyracksDataException e) {
-            LOGGER.log(Level.WARN, "Exception during ingestion", e);
+            logFailure(e);
             if (e.matches(ErrorCode.FEED_FAILED_WHILE_GETTING_A_NEW_RECORD)) {
                 // Failure but we know we can for sure push the previously parsed records safely
                 failure = e;
@@ -113,7 +113,7 @@
             }
         } catch (Throwable e) {
             failure = e;
-            LOGGER.log(Level.WARN, "Failure while operating a feed source", e);
+            logFailure(e);
         } finally {
             failure = finish(failure);
         }
@@ -126,7 +126,7 @@
     }
 
     private synchronized void setState(State newState) {
-        LOGGER.log(Level.INFO, "State is being set from " + state + " to " + newState);
+        LOGGER.info("controller is being set from {} to {} ", state, newState);
         state = newState;
     }
 
@@ -289,4 +289,12 @@
     public void handleGenericEvent(ActiveManagerMessage event) {
         recordReader.handleGenericEvent(event);
     }
+
+    private void logFailure(Throwable th) {
+        if (th instanceof InterruptedException || th.getCause() instanceof InterruptedException) {
+            LOGGER.warn("data flow controller interrupted", th);
+        } else {
+            LOGGER.warn("data flow controller failed", th);
+        }
+    }
 }
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 5da4583..4657bd0 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/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 050a080..479679e 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
@@ -334,108 +334,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 APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME = "applicationDefaultCredentials";
-        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 b61d62d..d730898 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,67 +18,23 @@
  */
 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_PARAM_VALUE_ALLOWED_VALUE;
-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.KEY_ADAPTER_NAME_GCS;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_DELIMITER;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ESCAPE;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_EXCLUDE;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_EXTERNAL_SCAN_BUFFER_SIZE;
-import static org.apache.asterix.external.util.ExternalDataConstants.KEY_FORMAT;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_INCLUDE;
 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.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.APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.GCS.JSON_CREDENTIALS_FIELD_NAME;
+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;
@@ -87,7 +43,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;
@@ -107,21 +62,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;
@@ -131,47 +90,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.GoogleCredentials;
-import com.google.cloud.NoCredentials;
-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;
@@ -471,7 +389,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);
@@ -493,8 +412,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)) {
@@ -516,7 +437,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
@@ -536,8 +458,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);
@@ -599,7 +523,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 {
@@ -607,16 +532,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
@@ -627,7 +552,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) {
@@ -642,7 +568,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) {
@@ -731,7 +658,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);
@@ -764,8 +692,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
@@ -849,8 +779,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 {
@@ -865,7 +797,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)
@@ -883,7 +815,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 {
@@ -902,7 +835,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)
@@ -914,1034 +848,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() != null && listObjectsResponse.isTruncated()) {
-                    newMarker = listObjectsResponse.nextContinuationToken();
-                } else {
-                    break;
-                }
-            }
-
-            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 applicationDefaultCredentials = configuration.get(APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME);
-            String jsonCredentials = configuration.get(JSON_CREDENTIALS_FIELD_NAME);
-            String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
-
-            StorageOptions.Builder builder = StorageOptions.newBuilder();
-
-            // default credentials provider
-            if (applicationDefaultCredentials != null) {
-                // only "true" value is allowed
-                if (!applicationDefaultCredentials.equalsIgnoreCase("true")) {
-                    throw new CompilationException(INVALID_PARAM_VALUE_ALLOWED_VALUE,
-                            APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME, "true");
-                }
-
-                // no other authentication parameters are allowed
-                if (jsonCredentials != null) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, JSON_CREDENTIALS_FIELD_NAME,
-                            APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME);
-                }
-
-                try {
-                    builder.setCredentials(GoogleCredentials.getApplicationDefault());
-                } catch (IOException ex) {
-                    throw CompilationException.create(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-                }
-            } else if (jsonCredentials != null) {
-                try (InputStream credentialsStream = new ByteArrayInputStream(jsonCredentials.getBytes())) {
-                    builder.setCredentials(GoogleCredentials.fromStream(credentialsStream));
-                } catch (IOException ex) {
-                    throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-                }
-            } else {
-                builder.setCredentials(NoCredentials.getInstance());
-            }
-
-            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);
-            }
-
-            // 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;
     }
@@ -1958,7 +864,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..ae32872
--- /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() != null && listObjectsResponse.isTruncated()) {
+                newMarker = listObjectsResponse.nextContinuationToken();
+            } else {
+                break;
+            }
+        }
+
+        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() != null && listObjectsResponse.isTruncated()) {
+                newMarker = listObjectsResponse.nextMarker();
+            } else {
+                break;
+            }
+        }
+
+        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..f2dbde7
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.google.gcs;
+
+public class GCSConstants {
+    private GCSConstants() {
+        throw new AssertionError("do not instantiate");
+    }
+
+    public static final String APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME = "applicationDefaultCredentials";
+    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..6183a88
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.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.external.util.ExternalDataUtils.getPrefix;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME;
+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.GoogleCredentials;
+import com.google.cloud.BaseServiceException;
+import com.google.cloud.NoCredentials;
+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 applicationDefaultCredentials = configuration.get(APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME);
+        String jsonCredentials = configuration.get(JSON_CREDENTIALS_FIELD_NAME);
+        String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
+
+        StorageOptions.Builder builder = StorageOptions.newBuilder();
+
+        // default credentials provider
+        if (applicationDefaultCredentials != null) {
+            // only "true" value is allowed
+            if (!applicationDefaultCredentials.equalsIgnoreCase("true")) {
+                throw new CompilationException(INVALID_PARAM_VALUE_ALLOWED_VALUE,
+                        APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME, "true");
+            }
+
+            // no other authentication parameters are allowed
+            if (jsonCredentials != null) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, JSON_CREDENTIALS_FIELD_NAME,
+                        APPLICATION_DEFAULT_CREDENTIALS_FIELD_NAME);
+            }
+
+            try {
+                builder.setCredentials(GoogleCredentials.getApplicationDefault());
+            } catch (IOException ex) {
+                throw CompilationException.create(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+            }
+        } else if (jsonCredentials != null) {
+            try (InputStream credentialsStream = new ByteArrayInputStream(jsonCredentials.getBytes())) {
+                builder.setCredentials(GoogleCredentials.fromStream(credentialsStream));
+            } catch (IOException ex) {
+                throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+            }
+        } else {
+            builder.setCredentials(NoCredentials.getInstance());
+        }
+
+        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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/asterixdb/asterix-external-data/src/main/java/org/checkerframework/checker/nullness/compatqual/NullableDecl.java
similarity index 61%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to asterixdb/asterix-external-data/src/main/java/org/checkerframework/checker/nullness/compatqual/NullableDecl.java
index 6afbccb..a4cb401 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/checkerframework/checker/nullness/compatqual/NullableDecl.java
@@ -16,17 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
+package org.checkerframework.checker.nullness.compatqual;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
 
-    void pause();
-
-    void resume();
+/*
+ * 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/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/util/LangRecordParseUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
index 0b9c895..dfe4c7f 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
@@ -61,7 +61,7 @@
     private LangRecordParseUtil() {
     }
 
-    private static IAdmNode parseExpression(Expression expr) throws HyracksDataException, CompilationException {
+    public static IAdmNode parseExpression(Expression expr) throws HyracksDataException, CompilationException {
         switch (expr.getKind()) {
             case LITERAL_EXPRESSION:
                 return parseLiteral((LiteralExpr) expr);
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/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/ILangVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
index b9de347..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;
@@ -222,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/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/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..aa54851 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..2bf0d21 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;
@@ -331,7 +332,12 @@
 
     @Override
     public Void visit(LimitClause limitClause, Collection<VariableExpr> freeVars) throws CompilationException {
-        limitClause.getLimitExpr().accept(this, freeVars);
+        if (limitClause.hasLimitExpr()) {
+            limitClause.getLimitExpr().accept(this, freeVars);
+        }
+        if (limitClause.hasOffset()) {
+            limitClause.getOffset().accept(this, freeVars);
+        }
         return null;
     }
 
@@ -468,6 +474,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 eda67bb..58810d1 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -4882,12 +4882,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()
   )?
@@ -4901,7 +4922,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;
   }
@@ -4959,7 +4980,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) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
index 0ab5c7b..254c372 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
@@ -39,6 +39,7 @@
 import org.apache.asterix.metadata.api.IAsterixStateProxy;
 import org.apache.asterix.metadata.api.IExtensionMetadataEntity;
 import org.apache.asterix.metadata.api.IExtensionMetadataSearchKey;
+import org.apache.asterix.metadata.api.IMetadataIndex;
 import org.apache.asterix.metadata.api.IMetadataManager;
 import org.apache.asterix.metadata.api.IMetadataNode;
 import org.apache.asterix.metadata.entities.CompactionPolicy;
@@ -64,6 +65,7 @@
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.google.common.base.Strings;
 
 /**
@@ -1192,6 +1194,16 @@
     }
 
     @Override
+    public JsonNode getEntitiesAsJson(MetadataTransactionContext mdTxnCtx, IMetadataIndex metadataIndex,
+            int payloadPosition) throws AlgebricksException {
+        try {
+            return metadataNode.getEntitiesAsJson(mdTxnCtx.getTxnId(), metadataIndex, payloadPosition);
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+    }
+
+    @Override
     public void rebindMetadataNode() {
         rebindMetadataNode = true;
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index 87f5129..7a6de19 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -22,6 +22,7 @@
 import static org.apache.asterix.common.api.IIdentifierMapper.Modifier.PLURAL;
 import static org.apache.asterix.common.utils.IdentifierUtil.dataset;
 
+import java.io.PrintStream;
 import java.rmi.RemoteException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -48,6 +49,7 @@
 import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.formats.nontagged.CleanJSONPrinterFactoryProvider;
 import org.apache.asterix.formats.nontagged.NullIntrospector;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.formats.nontagged.TypeTraitProvider;
@@ -119,11 +121,14 @@
 import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -142,9 +147,12 @@
 import org.apache.hyracks.storage.common.IModificationOperationCallback;
 import org.apache.hyracks.storage.common.MultiComparator;
 import org.apache.hyracks.util.ExitUtil;
+import org.apache.hyracks.util.JSONUtil;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
 import com.google.common.base.Strings;
 
 public class MetadataNode implements IMetadataNode {
@@ -327,6 +335,16 @@
         return getEntities(txnId, searchKey.getSearchKey(), tupleTranslator, index);
     }
 
+    @Override
+    public JsonNode getEntitiesAsJson(TxnId txnId, IMetadataIndex metadataIndex, int payloadPosition)
+            throws AlgebricksException, RemoteException {
+        try {
+            return getJsonNodes(txnId, metadataIndex, payloadPosition);
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
     private <T extends IExtensionMetadataEntity> ExtensionMetadataDataset<T> getExtensionMetadataDataset(
             ExtensionMetadataDatasetId datasetId) throws AlgebricksException {
         ExtensionMetadataDataset<T> index = (ExtensionMetadataDataset<T>) extensionDatasets.get(datasetId);
@@ -2485,4 +2503,44 @@
     public ITxnIdFactory getTxnIdFactory() {
         return txnIdFactory;
     }
+
+    private ArrayNode getJsonNodes(TxnId txnId, IMetadataIndex mdIndex, int payloadPosition)
+            throws AlgebricksException, HyracksDataException {
+        IValueExtractor<JsonNode> valueExtractor = createValueExtractor(mdIndex, payloadPosition);
+        List<JsonNode> results = new ArrayList<>();
+        searchIndex(txnId, mdIndex, null, valueExtractor, results);
+        ArrayNode array = JSONUtil.createArray();
+        results.forEach(array::add);
+        return array;
+    }
+
+    private static IValueExtractor<JsonNode> createValueExtractor(IMetadataIndex mdIndex, int payloadFieldIndex) {
+        return new IValueExtractor<>() {
+
+            final ARecordType payloadRecordType = mdIndex.getPayloadRecordType();
+            final IPrinterFactory printerFactory =
+                    CleanJSONPrinterFactoryProvider.INSTANCE.getPrinterFactory(payloadRecordType);
+            final IPrinter printer = printerFactory.createPrinter();
+            final ByteArrayAccessibleOutputStream outputStream = new ByteArrayAccessibleOutputStream();
+            final PrintStream printStream = new PrintStream(outputStream);
+
+            @Override
+            public JsonNode getValue(TxnId txnId, ITupleReference tuple) {
+                try {
+                    byte[] serRecord = tuple.getFieldData(payloadFieldIndex);
+                    int recordStartOffset = tuple.getFieldStart(payloadFieldIndex);
+                    int recordLength = tuple.getFieldLength(payloadFieldIndex);
+
+                    printer.init();
+                    outputStream.reset();
+
+                    printer.print(serRecord, recordStartOffset, recordLength, printStream);
+                    printStream.flush();
+                    return JSONUtil.readTree(outputStream.getByteArray(), 0, outputStream.getLength());
+                } catch (Throwable th) {
+                    return JSONUtil.createObject();
+                }
+            }
+        };
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
index 5ff8a03..d08578a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
@@ -46,6 +46,8 @@
 import org.apache.asterix.metadata.entities.Synonym;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 /**
  * A metadata manager provides user access to Asterix metadata (e.g., types,
  * datasets, indexes, etc.). A metadata manager satisfies requests by contacting
@@ -865,6 +867,20 @@
             IExtensionMetadataSearchKey searchKey) throws AlgebricksException;
 
     /**
+     * Gets all the records of a metadata dataset as JSON.
+     *
+     * @param mdTxnCtx metadata transaction context
+     * @param metadataIndex the metadata dataset
+     * @param payloadPosition the position of the record in the tuple
+     *
+     * @return the metadata records as JSON
+     *
+     * @throws AlgebricksException AlgebricksException
+     */
+    JsonNode getEntitiesAsJson(MetadataTransactionContext mdTxnCtx, IMetadataIndex metadataIndex, int payloadPosition)
+            throws AlgebricksException;
+
+    /**
      * Indicate when the metadata node has left or rejoined the cluster, and the
      * MetadataManager should rebind it
      */
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
index 561a4fa..378cf47 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
@@ -47,6 +47,8 @@
 import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 /**
  * A metadata node stores metadata in its local storage structures (currently
  * BTrees). A metadata node services requests on behalf of the (possibly remote)
@@ -969,6 +971,21 @@
     <T extends IExtensionMetadataEntity> List<T> getEntities(TxnId txnId, IExtensionMetadataSearchKey searchKey)
             throws AlgebricksException, RemoteException;
 
+    /**
+     * Gets all the records of a metadata dataset as JSON.
+     *
+     * @param txnId transaction id
+     * @param metadataIndex the metadata dataset
+     * @param payloadPosition the position of the record in the tuple
+     *
+     * @return the metadata records as JSON
+     *
+     * @throws AlgebricksException AlgebricksException
+     * @throws RemoteException RemoteException
+     */
+    JsonNode getEntitiesAsJson(TxnId txnId, IMetadataIndex metadataIndex, int payloadPosition)
+            throws AlgebricksException, RemoteException;
+
     void addFeedConnection(TxnId txnId, FeedConnection feedConnection) throws AlgebricksException, RemoteException;
 
     FeedConnection getFeedConnection(TxnId txnId, DataverseName dataverseName, String feedName, String datasetName)
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 66ea5a7..30f5783 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
@@ -55,6 +55,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 {
 
@@ -150,7 +151,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/FunctionDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
index 97c4c5d..3b6bd9d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
@@ -106,6 +106,10 @@
                 tupleFilterFactory, outputLimit);
     }
 
+    public boolean skipJobCapacityAssignment() {
+        return false;
+    }
+
     protected abstract IDatasourceFunction createFunction(MetadataProvider metadataProvider,
             AlgebricksAbsolutePartitionConstraint locations);
 
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 53cf3d0..dc2d017 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
@@ -174,6 +174,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> {
 
@@ -548,7 +549,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());
@@ -613,12 +615,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,
@@ -873,7 +876,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
index 885723c..8ed3e16 100644
--- 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
@@ -36,6 +36,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 SampleDataSource extends DataSource {
 
@@ -62,7 +63,7 @@
             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);
+                false, false, DefaultTupleProjectorFactory.INSTANCE);
     }
 
     @Override
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index 0d05d46..d502322 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -84,7 +84,7 @@
 public class DatasetTupleTranslator extends AbstractTupleTranslator<Dataset> {
 
     // Payload field containing serialized Dataset.
-    private static final int DATASET_PAYLOAD_TUPLE_FIELD_INDEX = 2;
+    public static final int DATASET_PAYLOAD_TUPLE_FIELD_INDEX = 2;
 
     protected AMutableInt32 aInt32;
     protected AMutableInt64 aInt64;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
index 77c838a..700beb9 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
@@ -39,7 +39,7 @@
 public class DataverseTupleTranslator extends AbstractTupleTranslator<Dataverse> {
 
     // Payload field containing serialized Dataverse.
-    private static final int DATAVERSE_PAYLOAD_TUPLE_FIELD_INDEX = 1;
+    public static final int DATAVERSE_PAYLOAD_TUPLE_FIELD_INDEX = 1;
 
     protected AMutableInt32 aInt32;
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
index ef9c143..9ad83c1 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
@@ -81,7 +81,7 @@
 public class FunctionTupleTranslator extends AbstractDatatypeTupleTranslator<Function> {
 
     // Payload field containing serialized Function.
-    private static final int FUNCTION_PAYLOAD_TUPLE_FIELD_INDEX = 3;
+    public static final int FUNCTION_PAYLOAD_TUPLE_FIELD_INDEX = 3;
 
     protected OrderedListBuilder dependenciesListBuilder;
     protected OrderedListBuilder dependencyListBuilder;
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 b00a706..08f947f 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
@@ -83,7 +83,7 @@
 public class IndexTupleTranslator extends AbstractTupleTranslator<Index> {
 
     // Payload field containing serialized Index.
-    private static final int INDEX_PAYLOAD_TUPLE_FIELD_INDEX = 3;
+    public static final int INDEX_PAYLOAD_TUPLE_FIELD_INDEX = 3;
 
     // Field name of open field.
     public static final String GRAM_LENGTH_FIELD_NAME = "GramLength";
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
index e3e879b..f255fff 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/LibraryTupleTranslator.java
@@ -45,7 +45,7 @@
 public class LibraryTupleTranslator extends AbstractTupleTranslator<Library> {
 
     // Payload field containing serialized Library.
-    private static final int LIBRARY_PAYLOAD_TUPLE_FIELD_INDEX = 2;
+    public static final int LIBRARY_PAYLOAD_TUPLE_FIELD_INDEX = 2;
 
     protected LibraryTupleTranslator(boolean getTuple) {
         super(getTuple, MetadataPrimaryIndexes.LIBRARY_DATASET, LIBRARY_PAYLOAD_TUPLE_FIELD_INDEX);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/SynonymTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/SynonymTupleTranslator.java
index 6bad2d4..12b410c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/SynonymTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/SynonymTupleTranslator.java
@@ -36,7 +36,7 @@
 
     // Payload field containing serialized Synonym.
 
-    private static final int SYNONYM_PAYLOAD_TUPLE_FIELD_INDEX = 2;
+    public static final int SYNONYM_PAYLOAD_TUPLE_FIELD_INDEX = 2;
 
     protected SynonymTupleTranslator(boolean getTuple) {
         super(getTuple, MetadataPrimaryIndexes.SYNONYM_DATASET, SYNONYM_PAYLOAD_TUPLE_FIELD_INDEX);
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-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 4db68c3..cdc8111 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
@@ -119,8 +119,12 @@
 import org.apache.asterix.om.typecomputer.impl.OrderedListOfAnyTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.PropagateTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.RecordAddFieldsTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.RecordAddTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.RecordMergeTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.RecordPutTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.RecordRemoveFieldsTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.RecordRemoveTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.RecordRenameTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ScalarArrayAggTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ScalarVersionOfAggregateResultType;
 import org.apache.asterix.om.typecomputer.impl.SleepTypeComputer;
@@ -250,6 +254,12 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-slice", 3);
     public static final FunctionIdentifier ARRAY_EXCEPT =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-except", 2);
+    public static final FunctionIdentifier ARRAY_SWAP =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-swap", 3);
+    public static final FunctionIdentifier ARRAY_MOVE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-move", 3);
+    public static final FunctionIdentifier ARRAY_BINARY_SEARCH =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-binary-search", 2);
 
     // objects
     public static final FunctionIdentifier RECORD_MERGE =
@@ -2408,6 +2418,9 @@
         addFunction(ARRAY_SLICE_WITH_END_POSITION, AListTypeComputer.INSTANCE_SLICE, true);
         addFunction(ARRAY_SLICE_WITHOUT_END_POSITION, AListTypeComputer.INSTANCE_SLICE, true);
         addFunction(ARRAY_EXCEPT, ArrayExceptTypeComputer.INSTANCE, true);
+        addFunction(ARRAY_MOVE, AListTypeComputer.INSTANCE_MOVE, true);
+        addFunction(ARRAY_SWAP, AListTypeComputer.INSTANCE_SWAP, true);
+        addFunction(ARRAY_BINARY_SEARCH, AInt32TypeComputer.INSTANCE_NULLABLE, true);
 
         // objects
         addFunction(RECORD_MERGE, RecordMergeTypeComputer.INSTANCE, true);
@@ -2428,12 +2441,12 @@
         addFunction(RECORD_PAIRS, OrderedListOfAnyTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(PAIRS, OrderedListOfAnyTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(GEOMETRY_CONSTRUCTOR, AGeometryTypeComputer.INSTANCE, true);
-        addFunction(RECORD_REMOVE, OpenARecordTypeComputer.INSTANCE, true);
-        addFunction(RECORD_RENAME, OpenARecordTypeComputer.INSTANCE, true);
+        addFunction(RECORD_REMOVE, RecordRemoveTypeComputer.INSTANCE, true);
+        addFunction(RECORD_RENAME, RecordRenameTypeComputer.INSTANCE, true);
         addFunction(RECORD_UNWRAP, AnyTypeComputer.INSTANCE, true);
         addFunction(RECORD_REPLACE, OpenARecordTypeComputer.INSTANCE, true);
-        addFunction(RECORD_ADD, OpenARecordTypeComputer.INSTANCE, true);
-        addFunction(RECORD_PUT, OpenARecordTypeComputer.INSTANCE, true);
+        addFunction(RECORD_ADD, RecordAddTypeComputer.INSTANCE, true);
+        addFunction(RECORD_PUT, RecordPutTypeComputer.INSTANCE, true);
         addFunction(RECORD_VALUES, OrderedListOfAnyTypeComputer.INSTANCE, true);
 
         // temporal type accessors
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/AListTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListTypeComputer.java
index e6e491c..21e88e2 100755
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListTypeComputer.java
@@ -41,6 +41,8 @@
     public static final AListTypeComputer INSTANCE_INSERT = new AListTypeComputer(3, false, true, false);
     public static final AListTypeComputer INSTANCE_REPLACE = new AListTypeComputer(3, false, true, false);
     public static final AListTypeComputer INSTANCE_SLICE = new AListTypeComputer(-1, false, false, true);
+    public static final AListTypeComputer INSTANCE_MOVE = new AListTypeComputer(3, false, false, true);
+    public static final AListTypeComputer INSTANCE_SWAP = new AListTypeComputer(3, false, false, true);
 
     private final int minNumArgs;
     private final boolean listIsLast;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractRecordFunctionTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractRecordFunctionTypeComputer.java
new file mode 100644
index 0000000..580cab4
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractRecordFunctionTypeComputer.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.om.typecomputer.impl;
+
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+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.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeHelper;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+/**
+ * Base type computer for the following record-functions:
+ * 1. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_ADD}
+ * 2. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_PUT}
+ * 3. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_REMOVE}
+ * 4. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_RENAME}
+ */
+public abstract class AbstractRecordFunctionTypeComputer implements IResultTypeComputer {
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        AbstractFunctionCallExpression functionCallExpression = (AbstractFunctionCallExpression) expression;
+
+        // Get our record type.
+        ILogicalExpression arg0 = functionCallExpression.getArguments().get(0).getValue();
+        IAType type0 = (IAType) env.getType(arg0);
+        IAType actualType0 = TypeComputeUtils.getActualType(type0);
+        ATypeTag tag0 = actualType0.getTypeTag();
+        if (tag0 == ATypeTag.ANY) {
+            return wrapTypeWithUnknown(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE, true, true);
+        } else if (tag0 == ATypeTag.MISSING) {
+            // Our output is always going to be MISSING.
+            return BuiltinType.AMISSING;
+        } else if (tag0 != ATypeTag.OBJECT) {
+            // Our output is always going to be NULL.
+            return BuiltinType.ANULL;
+        }
+        boolean isOutputMissable = TypeHelper.canBeMissing(type0);
+        boolean isOutputNullable = TypeHelper.canBeNull(type0);
+        ARecordType inputRecordType = TypeComputeUtils.extractRecordType(actualType0);
+
+        // Our second argument should be of type "string".
+        ILogicalExpression arg1 = functionCallExpression.getArguments().get(1).getValue();
+        IAType type1 = (IAType) env.getType(arg1);
+        IAType actualType1 = TypeComputeUtils.getActualType(type1);
+        ATypeTag tag1 = actualType1.getTypeTag();
+        if (tag1 == ATypeTag.ANY) {
+            // We cannot infer the type of our second argument-- our output may be MISSING or NULL.
+            return wrapTypeWithUnknown(type0, true, true);
+        } else if (tag1 == ATypeTag.MISSING) {
+            // Our output is always going to be MISSING.
+            return BuiltinType.AMISSING;
+        } else if (tag1 != ATypeTag.STRING) {
+            // Our output is always going to be NULL.
+            return BuiltinType.ANULL;
+        }
+        isOutputMissable |= TypeHelper.canBeMissing(type1);
+        isOutputNullable |= TypeHelper.canBeNull(type1);
+
+        // Compute our type.
+        return computeTypeImpl(functionCallExpression, env, inputRecordType, isOutputMissable, isOutputNullable);
+    }
+
+    protected abstract IAType computeTypeImpl(AbstractFunctionCallExpression functionCallExpression,
+            IVariableTypeEnvironment env, ARecordType inputRecordType, boolean isOutputMissable,
+            boolean isOutputNullable) throws AlgebricksException;
+
+    protected static IAType wrapTypeWithUnknown(IAType originalType, boolean isMissable, boolean isNullable) {
+        if (isNullable && isMissable) {
+            return AUnionType.createUnknownableType(originalType);
+        } else if (isNullable) { // && !isMissable
+            return AUnionType.createNullableType(originalType);
+        } else if (isMissable) { // && !isNullable
+            return AUnionType.createMissableType(originalType);
+        } else {
+            return originalType;
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddTypeComputer.java
new file mode 100644
index 0000000..4815340
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddTypeComputer.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.asterix.om.typecomputer.impl;
+
+import java.util.Arrays;
+
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+
+public class RecordAddTypeComputer extends AbstractRecordFunctionTypeComputer {
+    public static final RecordAddTypeComputer INSTANCE = new RecordAddTypeComputer();
+
+    private RecordAddTypeComputer() {
+    }
+
+    @Override
+    public IAType computeTypeImpl(AbstractFunctionCallExpression functionCallExpression, IVariableTypeEnvironment env,
+            ARecordType inputRecordType, boolean isOutputMissable, boolean isOutputNullable)
+            throws AlgebricksException {
+        // If our third argument is missing, then just return the type of our first argument.
+        ILogicalExpression arg2 = functionCallExpression.getArguments().get(2).getValue();
+        IAType type2 = (IAType) env.getType(arg2);
+        IAType actualType2 = TypeComputeUtils.getActualType(type2);
+        ATypeTag tag2 = actualType2.getTypeTag();
+        if (tag2 == ATypeTag.MISSING) {
+            IAType type0 = (IAType) env.getType(functionCallExpression.getArguments().get(0).getValue());
+            return wrapTypeWithUnknown(type0, isOutputMissable, isOutputNullable);
+        }
+
+        // We expect a constant for our second argument.
+        ILogicalExpression arg1 = functionCallExpression.getArguments().get(1).getValue();
+        if (arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            return wrapTypeWithUnknown(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE, isOutputMissable,
+                    isOutputNullable);
+        }
+        ConstantExpression constantExpression = (ConstantExpression) arg1;
+        AsterixConstantValue constantValue = (AsterixConstantValue) constantExpression.getValue();
+        String newFieldName = ((AString) constantValue.getObject()).getStringValue();
+
+        // If our field is found, return the original record type. Otherwise, add our new field.
+        ARecordType outputRecordType = inputRecordType;
+        if (!Arrays.asList(inputRecordType.getFieldNames()).contains(newFieldName)) {
+            String[] fieldNames = new String[inputRecordType.getFieldNames().length + 1];
+            IAType[] fieldTypes = new IAType[inputRecordType.getFieldTypes().length + 1];
+            int currentCursorPosition = 0;
+            for (; currentCursorPosition < inputRecordType.getFieldNames().length; currentCursorPosition++) {
+                fieldNames[currentCursorPosition] = inputRecordType.getFieldNames()[currentCursorPosition];
+                fieldTypes[currentCursorPosition] = inputRecordType.getFieldTypes()[currentCursorPosition];
+            }
+            fieldNames[currentCursorPosition] = newFieldName;
+            fieldTypes[currentCursorPosition] = type2;
+            String inputTypeName = inputRecordType.getTypeName();
+            String outputTypeName = inputTypeName != null ? inputTypeName + "_add_" + newFieldName : null;
+            outputRecordType = new ARecordType(outputTypeName, fieldNames, fieldTypes, inputRecordType.isOpen());
+        }
+        return wrapTypeWithUnknown(outputRecordType, isOutputMissable, isOutputNullable);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordPutTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordPutTypeComputer.java
new file mode 100644
index 0000000..a92907d
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordPutTypeComputer.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+
+public class RecordPutTypeComputer extends AbstractRecordFunctionTypeComputer {
+    public static final RecordPutTypeComputer INSTANCE = new RecordPutTypeComputer();
+
+    private RecordPutTypeComputer() {
+    }
+
+    @Override
+    public IAType computeTypeImpl(AbstractFunctionCallExpression functionCallExpression, IVariableTypeEnvironment env,
+            ARecordType inputRecordType, boolean isOutputMissable, boolean isOutputNullable)
+            throws AlgebricksException {
+        // Extract the type of our third argument. If it is MISSING, then we are performing a field removal.
+        ILogicalExpression arg2 = functionCallExpression.getArguments().get(2).getValue();
+        IAType type2 = (IAType) env.getType(arg2);
+        IAType actualType2 = TypeComputeUtils.getActualType(type2);
+        boolean isFieldRemoval = actualType2.getTypeTag() == ATypeTag.MISSING;
+
+        // We expect a constant for our second argument.
+        ILogicalExpression arg1 = functionCallExpression.getArguments().get(1).getValue();
+        if (arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            return wrapTypeWithUnknown(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE, isOutputMissable,
+                    isOutputNullable);
+        }
+        ConstantExpression constantExpression = (ConstantExpression) arg1;
+        AsterixConstantValue constantValue = (AsterixConstantValue) constantExpression.getValue();
+        String newFieldName = ((AString) constantValue.getObject()).getStringValue();
+
+        // Remove or replace our field name and type (dependent on the type of our third argument).
+        boolean fieldFound = false;
+        List<String> outputFieldNames = new ArrayList<>();
+        List<IAType> outputFieldTypes = new ArrayList<>();
+        for (int i = 0; i < inputRecordType.getFieldNames().length; i++) {
+            String inputFieldName = inputRecordType.getFieldNames()[i];
+            IAType inputFieldType = inputRecordType.getFieldTypes()[i];
+            if (!inputFieldName.equals(newFieldName)) {
+                outputFieldNames.add(inputFieldName);
+                outputFieldTypes.add(inputFieldType);
+
+            } else {
+                fieldFound = true;
+                if (!isFieldRemoval) {
+                    // Replace our input field type.
+                    outputFieldNames.add(inputFieldName);
+                    outputFieldTypes.add(type2);
+                }
+            }
+        }
+
+        // Build our output record type.
+        ARecordType outputRecordType;
+        String inputTypeName = inputRecordType.getTypeName();
+        boolean doesRecordHaveTypeName = inputTypeName != null;
+        if (fieldFound && isFieldRemoval) {
+            // We have removed our argument field.
+            String outputTypeName = doesRecordHaveTypeName ? inputTypeName + "_remove_" + newFieldName : null;
+            outputRecordType = new ARecordType(outputTypeName, outputFieldNames.toArray(String[]::new),
+                    outputFieldTypes.toArray(IAType[]::new), inputRecordType.isOpen());
+        } else if (fieldFound) { // && !isFieldRemoval
+            // We have replaced our argument field.
+            String outputTypeName = doesRecordHaveTypeName ? inputTypeName + "_replaced_" + newFieldName : null;
+            outputRecordType = new ARecordType(outputTypeName, outputFieldNames.toArray(String[]::new),
+                    outputFieldTypes.toArray(IAType[]::new), inputRecordType.isOpen());
+        } else if (!isFieldRemoval) { // && !wasFieldFound
+            // We need to insert our argument field.
+            outputFieldNames.add(newFieldName);
+            outputFieldTypes.add(type2);
+            String outputTypeName = doesRecordHaveTypeName ? inputTypeName + "_add_" + newFieldName : null;
+            outputRecordType = new ARecordType(outputTypeName, outputFieldNames.toArray(String[]::new),
+                    outputFieldTypes.toArray(IAType[]::new), inputRecordType.isOpen());
+        } else { // isFieldRemoval && !wasFieldFound
+            // We have not found the field to remove.
+            outputRecordType = inputRecordType;
+        }
+        return wrapTypeWithUnknown(outputRecordType, isOutputMissable, isOutputNullable);
+    }
+}
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-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveTypeComputer.java
new file mode 100644
index 0000000..c447e65
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveTypeComputer.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import java.util.Arrays;
+
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+
+public class RecordRemoveTypeComputer extends AbstractRecordFunctionTypeComputer {
+    public static final RecordRemoveTypeComputer INSTANCE = new RecordRemoveTypeComputer();
+
+    private RecordRemoveTypeComputer() {
+    }
+
+    @Override
+    public IAType computeTypeImpl(AbstractFunctionCallExpression functionCallExpression, IVariableTypeEnvironment env,
+            ARecordType inputRecordType, boolean isOutputMissable, boolean isOutputNullable) {
+        // We expect a CONSTANT expression. Otherwise, defer the removal to runtime.
+        ILogicalExpression arg1 = functionCallExpression.getArguments().get(1).getValue();
+        if (arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            return wrapTypeWithUnknown(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE, isOutputMissable,
+                    isOutputNullable);
+        }
+        ConstantExpression constantExpression = (ConstantExpression) arg1;
+        AsterixConstantValue constantValue = (AsterixConstantValue) constantExpression.getValue();
+        String fieldName = ((AString) constantValue.getObject()).getStringValue();
+
+        // If our field is found, remove it. Otherwise, return the original record type.
+        ARecordType outputRecordType = inputRecordType;
+        if (Arrays.asList(inputRecordType.getFieldNames()).contains(fieldName)) {
+            String[] fieldNames = new String[inputRecordType.getFieldNames().length - 1];
+            IAType[] fieldTypes = new IAType[inputRecordType.getFieldTypes().length - 1];
+            int currentOutputCursor = 0;
+            for (int i = 0; i < inputRecordType.getFieldNames().length; i++) {
+                String inputName = inputRecordType.getFieldNames()[i];
+                IAType inputType = inputRecordType.getFieldTypes()[i];
+                if (!inputName.equals(fieldName)) {
+                    fieldNames[currentOutputCursor] = inputName;
+                    fieldTypes[currentOutputCursor] = inputType;
+                    currentOutputCursor++;
+                }
+            }
+            String inputTypeName = inputRecordType.getTypeName();
+            String outputTypeName = inputTypeName != null ? inputTypeName + "_remove_" + fieldName : null;
+            outputRecordType = new ARecordType(outputTypeName, fieldNames, fieldTypes, inputRecordType.isOpen());
+        }
+        return wrapTypeWithUnknown(outputRecordType, isOutputMissable, isOutputNullable);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRenameTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRenameTypeComputer.java
new file mode 100644
index 0000000..4302082
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRenameTypeComputer.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import java.util.Arrays;
+
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+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.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.commons.lang3.mutable.MutableObject;
+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.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+
+public class RecordRenameTypeComputer extends AbstractRecordFunctionTypeComputer {
+    public static final RecordRenameTypeComputer INSTANCE = new RecordRenameTypeComputer();
+
+    private RecordRenameTypeComputer() {
+    }
+
+    @Override
+    public IAType computeTypeImpl(AbstractFunctionCallExpression functionCallExpression, IVariableTypeEnvironment env,
+            ARecordType inputRecordType, boolean isOutputMissable, boolean isOutputNullable)
+            throws AlgebricksException {
+        // Our third argument should be of type "string".
+        ILogicalExpression arg2 = functionCallExpression.getArguments().get(2).getValue();
+        IAType type2 = (IAType) env.getType(arg2);
+        IAType actualType2 = TypeComputeUtils.getActualType(type2);
+        ATypeTag tag2 = actualType2.getTypeTag();
+        if (tag2 == ATypeTag.ANY) {
+            // We cannot infer the type of our third argument-- our output may be MISSING or NULL.
+            return AUnionType.createUnknownableType(inputRecordType, inputRecordType.getTypeName() + "?");
+        } else if (tag2 == ATypeTag.MISSING) {
+            // Our output is always going to be MISSING.
+            return BuiltinType.AMISSING;
+        } else if (tag2 != ATypeTag.STRING) {
+            // Our output is always going to be NULL.
+            return BuiltinType.ANULL;
+        }
+        isOutputMissable |= TypeHelper.canBeMissing(type2);
+        isOutputNullable |= TypeHelper.canBeNull(type2);
+
+        // We expect a CONSTANT expression for both arguments. Otherwise, defer the replacement to runtime.
+        ILogicalExpression arg1 = functionCallExpression.getArguments().get(1).getValue();
+        if (arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT
+                || arg2.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            return wrapTypeWithUnknown(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE, isOutputMissable,
+                    isOutputNullable);
+        }
+        ConstantExpression arg1ConstantExpression = (ConstantExpression) arg1;
+        ConstantExpression arg2ConstantExpression = (ConstantExpression) arg2;
+        AsterixConstantValue arg1ConstantValue = (AsterixConstantValue) arg1ConstantExpression.getValue();
+        AsterixConstantValue arg2ConstantValue = (AsterixConstantValue) arg2ConstantExpression.getValue();
+        String oldFieldName = ((AString) arg1ConstantValue.getObject()).getStringValue();
+        String newFieldName = ((AString) arg2ConstantValue.getObject()).getStringValue();
+
+        // If our field is found, replace it. Otherwise, return the original record type.
+        Mutable<Boolean> fieldFound = new MutableObject<>(false);
+        String[] newFieldNames = Arrays.stream(inputRecordType.getFieldNames()).map(f -> {
+            if (f.equals(oldFieldName)) {
+                fieldFound.setValue(true);
+                return newFieldName;
+            } else {
+                return f;
+            }
+        }).toArray(String[]::new);
+        ARecordType outputRecordType;
+        if (!fieldFound.getValue()) {
+            outputRecordType = inputRecordType;
+        } else {
+            String inputTypeName = inputRecordType.getTypeName();
+            String outputTypeName = inputTypeName != null ? inputTypeName + "_replaced_" + oldFieldName : null;
+            outputRecordType = new ARecordType(outputTypeName, newFieldNames, inputRecordType.getFieldTypes(),
+                    inputRecordType.isOpen());
+        }
+        return wrapTypeWithUnknown(outputRecordType, isOutputMissable, isOutputNullable);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
index e256e1b..fd51433 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
@@ -25,7 +25,6 @@
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Objects;
 import java.util.Set;
 
 import org.apache.asterix.common.annotations.IRecordTypeAnnotation;
@@ -73,30 +72,21 @@
     private final Set<String> allPossibleAdditionalFieldNames;
 
     /**
-     * @param typeName
-     *            the name of the type
-     * @param fieldNames
-     *            the names of the closed fields
-     * @param fieldTypes
-     *            the types of the closed fields
-     * @param isOpen
-     *            whether the record is open
+     * @param typeName   the name of the type
+     * @param fieldNames the names of the closed fields
+     * @param fieldTypes the types of the closed fields
+     * @param isOpen     whether the record is open
      */
     public ARecordType(String typeName, String[] fieldNames, IAType[] fieldTypes, boolean isOpen) {
         this(typeName, fieldNames, fieldTypes, isOpen, null);
     }
 
     /**
-     * @param typeName
-     *            the name of the type
-     * @param fieldNames
-     *            the names of the closed fields
-     * @param fieldTypes
-     *            the types of the closed fields
-     * @param isOpen
-     *            whether the record is open
-     * @param allPossibleAdditionalFieldNames,
-     *            all possible additional field names.
+     * @param typeName                         the name of the type
+     * @param fieldNames                       the names of the closed fields
+     * @param fieldTypes                       the types of the closed fields
+     * @param isOpen                           whether the record is open
+     * @param allPossibleAdditionalFieldNames, all possible additional field names.
      */
     public ARecordType(String typeName, String[] fieldNames, IAType[] fieldTypes, boolean isOpen,
             Set<String> allPossibleAdditionalFieldNames) {
@@ -189,8 +179,7 @@
     /**
      * Returns the position of the field in the closed schema or -1 if the field does not exist.
      *
-     * @param fieldName
-     *            the name of the field whose position is sought
+     * @param fieldName the name of the field whose position is sought
      * @return the position of the field in the closed schema or -1 if the field does not exist.
      */
     public int getFieldIndex(String fieldName) {
@@ -206,10 +195,8 @@
     }
 
     /**
-     * @param subFieldName
-     *            The full pathname of the child
-     * @param parent
-     *            The type of the parent
+     * @param subFieldName The full pathname of the child
+     * @param parent       The type of the parent
      * @return the type of the child
      */
 
@@ -222,8 +209,7 @@
     }
 
     /**
-     * @param subFieldName
-     *            The full pathname of the child
+     * @param subFieldName The full pathname of the child
      * @return the type of the child
      * @throws AsterixException
      */
@@ -248,9 +234,7 @@
     }
 
     /**
-     *
-     * @param subFieldName
-     *            The full pathname of the field
+     * @param subFieldName The full pathname of the field
      * @return The nullability of the field
      * @throws AlgebricksException
      */
@@ -283,10 +267,9 @@
     /**
      * Returns the field type of the field name if it exists, otherwise null.
      *
-     * @param fieldName
-     *            the fieldName whose type is sought
+     * @param fieldName the fieldName whose type is sought
      * @return the field type of the field name if it exists, otherwise null
-     *         NOTE: this method doesn't work for nested fields
+     * NOTE: this method doesn't work for nested fields
      */
     public IAType getFieldType(String fieldName) {
         int fieldPos = getFieldIndex(fieldName);
@@ -299,8 +282,7 @@
     /**
      * Returns true or false indicating whether or not a field is closed.
      *
-     * @param fieldName
-     *            the name of the field to check
+     * @param fieldName the name of the field to check
      * @return true if fieldName is a closed field, otherwise false
      */
     public boolean isClosedField(String fieldName) {
@@ -360,8 +342,7 @@
         }
         ARecordType rt = (ARecordType) obj;
         return (isOpen == rt.isOpen) && Arrays.deepEquals(fieldNames, rt.fieldNames)
-                && Arrays.deepEquals(fieldTypes, rt.fieldTypes)
-                && Objects.equals(allPossibleAdditionalFieldNames, rt.allPossibleAdditionalFieldNames);
+                && Arrays.deepEquals(fieldTypes, rt.fieldTypes);
     }
 
     @Override
@@ -382,11 +363,7 @@
         ObjectNode type = om.createObjectNode();
         type.put("type", ARecordType.class.getName());
         type.put("name", typeName);
-        if (isOpen) {
-            type.put("open", true);
-        } else {
-            type.put("open", false);
-        }
+        type.put("open", isOpen);
 
         ArrayNode fields = om.createArrayNode();
         for (int i = 0; i < fieldNames.length; 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..5fe3b7c
--- /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.2-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-replication/src/main/java/org/apache/asterix/replication/management/NetworkingUtil.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/NetworkingUtil.java
index b38f0aa..9caaa79 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/NetworkingUtil.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/NetworkingUtil.java
@@ -21,18 +21,14 @@
 import java.io.EOFException;
 import java.io.IOException;
 import java.net.InetAddress;
-import java.net.InetSocketAddress;
 import java.net.NetworkInterface;
-import java.net.SocketAddress;
 import java.net.SocketException;
-import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.nio.MappedByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.channels.SocketChannel;
 import java.util.Enumeration;
 
-import org.apache.hyracks.api.comm.NetworkAddress;
 import org.apache.hyracks.api.network.ISocketChannel;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -131,16 +127,6 @@
         fileChannel.transferFrom(socketChannel, pos, fileSize);
     }
 
-    public static InetSocketAddress getSocketAddress(SocketChannel socketChannel) {
-        String hostAddress = socketChannel.socket().getInetAddress().getHostAddress();
-        int port = socketChannel.socket().getPort();
-        return InetSocketAddress.createUnresolved(hostAddress, port);
-    }
-
-    public static SocketAddress getSocketAddress(NetworkAddress netAddr) throws UnknownHostException {
-        return new InetSocketAddress(InetAddress.getByAddress(netAddr.lookupIpAddress()), netAddr.getPort());
-    }
-
     public static boolean isHealthy(ISocketChannel sc) {
         return sc != null && sc.getSocketChannel().isOpen() && sc.getSocketChannel().isConnected();
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ClosedRecordConstructorEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ClosedRecordConstructorEvalFactory.java
index 2347fa5..f22f5da 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ClosedRecordConstructorEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ClosedRecordConstructorEvalFactory.java
@@ -46,6 +46,11 @@
     }
 
     @Override
+    public String toString() {
+        return "ClosedRecordConstructor";
+    }
+
+    @Override
     public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
         int n = args.length / 2;
         IScalarEvaluator[] evalFields = new IScalarEvaluator[n];
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArrayMoveSwapEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArrayMoveSwapEval.java
new file mode 100644
index 0000000..218e5ac
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArrayMoveSwapEval.java
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions;
+
+import static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+import java.io.IOException;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeTagUtil;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+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.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * Abstract class for the ARRAY_MOVE and ARRAY_SWAP functions as they share a large portion of their code
+ * with each other. Any classes that use this abstract one should override the `buildList` method, as it just
+ * otherwise builds the same input list.
+ */
+public abstract class AbstractArrayMoveSwapEval implements IScalarEvaluator {
+
+    private final ArrayBackedValueStorage storage;
+    private final IScalarEvaluator listArgEval;
+    private final IScalarEvaluator oldIndexEval;
+    private final IScalarEvaluator newIndexEval;
+    private final IPointable listArg;
+    private final IPointable oldIndex;
+    private final IPointable newIndex;
+    private final ListAccessor listAccessor;
+    private IAsterixListBuilder listBuilder;
+    private String funcIdentifier;
+    private IAType inputListType;
+
+    AbstractArrayMoveSwapEval(IScalarEvaluatorFactory[] args, IEvaluatorContext ctx, String funcIdentifier,
+            IAType inputListType) throws HyracksDataException {
+
+        this.funcIdentifier = funcIdentifier;
+        this.inputListType = inputListType;
+        storage = new ArrayBackedValueStorage();
+        listArgEval = args[0].createScalarEvaluator(ctx);
+        oldIndexEval = args[1].createScalarEvaluator(ctx);
+        newIndexEval = args[2].createScalarEvaluator(ctx);
+        listArg = new VoidPointable();
+        oldIndex = new VoidPointable();
+        newIndex = new VoidPointable();
+        listAccessor = new ListAccessor();
+        listBuilder = new OrderedListBuilder();
+    }
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+
+        storage.reset();
+
+        // Check that our args aren't missing/null
+        listArgEval.evaluate(tuple, listArg);
+        oldIndexEval.evaluate(tuple, oldIndex);
+        newIndexEval.evaluate(tuple, newIndex);
+        if (PointableHelper.checkAndSetMissingOrNull(result, listArg, oldIndex, newIndex)) {
+            return;
+        }
+
+        byte[] listBytes = listArg.getByteArray();
+        int offset = listArg.getStartOffset();
+        ATypeTag listType = ATYPETAGDESERIALIZER.deserialize(listBytes[offset]);
+
+        byte[] oldIndexBytes = oldIndex.getByteArray();
+        int oldIndexOffset = oldIndex.getStartOffset();
+        ATypeTag oldIndexType = ATYPETAGDESERIALIZER.deserialize((oldIndexBytes[oldIndexOffset]));
+
+        byte[] newIndexBytes = newIndex.getByteArray();
+        int newIndexOffset = newIndex.getStartOffset();
+        ATypeTag newIndexType = ATYPETAGDESERIALIZER.deserialize(newIndexBytes[newIndexOffset]);
+
+        // Checks that the list is of ordered list type, and that the two indices are valid numeric values.
+        // e.g) 1.0, 2, 4.0 works, but 4.5, 3.2 would not.
+        if (!(listType == ATypeTag.ARRAY) || !ATypeHierarchy.isCompatible(oldIndexType, ATypeTag.DOUBLE)
+                || !ATypeHierarchy.isCompatible(newIndexType, ATypeTag.DOUBLE)) {
+            PointableHelper.setNull(result);
+            return;
+        }
+
+        listAccessor.reset(listBytes, offset);
+
+        AbstractCollectionType outputListType;
+
+        ATypeTag listItemTypeTag = listAccessor.getItemType();
+        IAType listItemType = TypeTagUtil.getBuiltinTypeByTag(listItemTypeTag);
+        if (listAccessor.getListType() == ATypeTag.ARRAY) {
+            outputListType = new AOrderedListType(listItemType, listItemType.getTypeName());
+        }
+        // Known list type, use it directly
+        else {
+            outputListType = (AbstractCollectionType) inputListType;
+        }
+
+        listBuilder.reset(outputListType);
+
+        try {
+
+            int listLen = listAccessor.size();
+
+            double oldIndexVal = ATypeHierarchy.getDoubleValue(funcIdentifier, 1, oldIndexBytes, oldIndexOffset);
+            double newIndexVal = ATypeHierarchy.getDoubleValue(funcIdentifier, 2, newIndexBytes, newIndexOffset);
+
+            //Checks that old/new indices are within the range of the list and whether they are valid values
+            if (Double.isNaN(oldIndexVal) || Double.isInfinite(oldIndexVal) || Math.floor(oldIndexVal) < oldIndexVal
+                    || newIndexVal > (listLen - 1) || newIndexVal < -(listLen) || oldIndexVal < -(listLen)
+                    || oldIndexVal > (listLen - 1)) {
+                PointableHelper.setNull(result);
+                return;
+            }
+
+            // Converting the indices values into integers to be used in iteration. Also accounting for the negative indices case by using modulo
+            int oldIndexInt = (int) oldIndexVal;
+            int newIndexInt = (int) newIndexVal;
+
+            // use modulus to account for negative indices case
+            oldIndexInt = (oldIndexInt + listLen) % listLen;
+            newIndexInt = (newIndexInt + listLen) % listLen;
+
+            // if no changes are to be made, then return original list
+            if (oldIndexInt == newIndexInt || listLen <= 1) {
+                result.set(listArg);
+                return;
+            }
+
+            buildList(oldIndexInt, newIndexInt, listLen, listAccessor, listBuilder);
+
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        storage.reset();
+        listBuilder.write(storage.getDataOutput(), true);
+        result.set(storage);
+    }
+
+    /**
+     *
+     * Default: Builds a list with the exact same values as the input list. Depending on the function that extends
+     * this abstract class, this method should be overridden to suit the extending function.
+     *
+     * @param oldIndexInt - Position of the item at the old index.
+     * @param newIndexInt - Position where the item at the old index wants to be.
+     * @param listLen - to iterate through the list
+     * @param listAccessor
+     * @param listBuilder
+     * @throws IOException
+     */
+    protected void buildList(int oldIndexInt, int newIndexInt, int listLen, ListAccessor listAccessor,
+            IAsterixListBuilder listBuilder) throws IOException {
+        for (int i = 0; i < listLen; i++) {
+            storage.reset();
+            listAccessor.writeItem(oldIndexInt, storage.getDataOutput());
+            listBuilder.addItem(storage);
+        }
+
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayBinarySearchDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayBinarySearchDescriptor.java
new file mode 100644
index 0000000..23e13d4
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayBinarySearchDescriptor.java
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions;
+
+import static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+import java.io.IOException;
+
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+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.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.asterix.runtime.utils.DescriptorFactoryUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+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.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * array_binary_search(orderedList, searchValue) returns the index of the search value if it exists within the
+ * ordered list.
+ *
+ * It returns in order:
+ * Missing, if any of the arguments are missing.
+ * Null, if the arguments are null, if the list argument is not a list, or if the searchValue argument is not numerical.
+ * Otherwise, it returns the index of the first occurrence of the search value in the input list.
+ */
+
+public class ArrayBinarySearchDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    private IAType[] argTypes;
+
+    public static final IFunctionDescriptorFactory FACTORY = DescriptorFactoryUtil
+            .createFactory(ArrayBinarySearchDescriptor::new, FunctionTypeInferers.SET_ARGUMENTS_TYPE);
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        argTypes = (IAType[]) states;
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_BINARY_SEARCH;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
+                return new ArrayBinarySearchDescriptor.ArrayBinarySearchEval(args, ctx, argTypes);
+            }
+        };
+    }
+
+    public class ArrayBinarySearchEval implements IScalarEvaluator {
+
+        private final ArrayBackedValueStorage storage;
+        private final ArrayBackedValueStorage tempStorage;
+        private final IScalarEvaluator listArgEval;
+        private final IScalarEvaluator searchArgEval;
+        private final IPointable listArg;
+        private final IPointable searchArg;
+        private final IPointable tempVal;
+        private final IPointable tempVal2;
+        private final ListAccessor listAccessor;
+        private final IBinaryComparator comp;
+        private final ISerializerDeserializer<AInt32> serde =
+                SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+        private final AMutableInt32 resIndex = new AMutableInt32(0);
+
+        public ArrayBinarySearchEval(IScalarEvaluatorFactory[] args, IEvaluatorContext ctx, IAType[] argTypes)
+                throws HyracksDataException {
+            storage = new ArrayBackedValueStorage();
+            tempStorage = new ArrayBackedValueStorage();
+            listArg = new VoidPointable();
+            searchArg = new VoidPointable();
+            tempVal = new VoidPointable();
+            tempVal2 = new VoidPointable();
+            listArgEval = args[0].createScalarEvaluator(ctx);
+            searchArgEval = args[1].createScalarEvaluator(ctx);
+            listAccessor = new ListAccessor();
+            comp = createComparator(argTypes[0], argTypes[1]);
+        }
+
+        private IBinaryComparator createComparator(IAType listType, IAType searchValueType) {
+            IAType itemType = listType.getTypeTag().isListType() ? ((AbstractCollectionType) listType).getItemType()
+                    : BuiltinType.ANY;
+            return BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(itemType, searchValueType, true)
+                    .createBinaryComparator();
+        }
+
+        @Override
+        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+
+            // argument missing/null checks
+            listArgEval.evaluate(tuple, listArg);
+            searchArgEval.evaluate(tuple, searchArg);
+            if (PointableHelper.checkAndSetMissingOrNull(result, listArg, searchArg)) {
+                return;
+            }
+
+            // Checking that our list arg is in fact a list
+            byte[] listBytes = listArg.getByteArray();
+            int offset = listArg.getStartOffset();
+            ATypeTag listType = ATYPETAGDESERIALIZER.deserialize(listBytes[offset]);
+
+            if (listType != ATypeTag.ARRAY) {
+                PointableHelper.setNull(result);
+                return;
+            }
+
+            byte[] searchBytes = searchArg.getByteArray();
+            int searchOffset = searchArg.getStartOffset();
+
+            listAccessor.reset(listBytes, offset);
+
+            int listLen = listAccessor.size();
+            int low = 0;
+            int high = listLen - 1;
+
+            try {
+                while (low <= high) {
+                    int mid = low + ((high - low) / 2);
+                    storage.reset();
+                    listAccessor.getOrWriteItem(mid, tempVal, storage);
+                    int comparison = comp.compare(tempVal.getByteArray(), tempVal.getStartOffset(), tempVal.getLength(),
+                            searchBytes, searchOffset, searchArg.getLength());
+                    if (comparison == 0) {
+                        // if found, then find the first occurrence of the searchValue (from left to right)
+                        int firstFoundIndex =
+                                fetchFirstValue(mid, storage, tempStorage, listAccessor, comp, tempVal, tempVal2);
+                        storage.reset();
+                        resIndex.setValue(firstFoundIndex);
+                        serde.serialize(resIndex, storage.getDataOutput());
+                        result.set(storage);
+                        return;
+                    } else if (comparison < 0) {
+                        low = mid + 1;
+                    } else {
+                        high = mid - 1;
+                    }
+                }
+                storage.reset();
+                resIndex.setValue(-1);
+                serde.serialize(resIndex, storage.getDataOutput());
+                result.set(storage);
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            }
+        }
+    }
+
+    private int fetchFirstValue(int midIndexArg, ArrayBackedValueStorage storage, ArrayBackedValueStorage storage2,
+            ListAccessor listAccessor, IBinaryComparator comp, IPointable tempVal1, IPointable tempVal2)
+            throws IOException {
+
+        int midIndex = midIndexArg;
+
+        if (midIndex == 0) {
+            return midIndex;
+        }
+        storage.reset();
+        listAccessor.getOrWriteItem(midIndex, tempVal1, storage);
+        storage.reset();
+        listAccessor.getOrWriteItem(midIndex - 1, tempVal2, storage2);
+        int prevComparison = comp.compare(tempVal1.getByteArray(), tempVal1.getStartOffset(), tempVal1.getLength(),
+                tempVal2.getByteArray(), tempVal2.getStartOffset(), tempVal2.getLength());
+        // If values before current value are not equal, then return current position.
+        if (prevComparison != 0) {
+            return midIndex;
+        } else {
+            // midIndex-1 position was already checked, so we now start checking the previous positions
+            midIndex--;
+            // to count the number of positions before the "midIndex" value to find first occurrence of search value.
+            int counter = 0;
+            while (prevComparison == 0) {
+                counter++;
+                if (midIndex - counter == 0) {
+                    return 0;
+                }
+                storage2.reset();
+                listAccessor.getOrWriteItem(midIndex - counter, tempVal2, storage2);
+                prevComparison = comp.compare(tempVal1.getByteArray(), tempVal1.getStartOffset(), tempVal1.getLength(),
+                        tempVal2.getByteArray(), tempVal2.getStartOffset(), tempVal2.getLength());
+                if (prevComparison != 0) {
+                    return (midIndex - counter + 1);
+                }
+            }
+        }
+        return -1;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayMoveDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayMoveDescriptor.java
new file mode 100644
index 0000000..82b5465
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayMoveDescriptor.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions;
+
+import java.io.IOException;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.asterix.runtime.utils.DescriptorFactoryUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+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.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+/**
+ * array_move(list, position1, position2) returns a new list, moving the item in position1 in such a way
+ * that now, it will be in position2. It will also move all other items accordingly.
+ *
+ * It returns in order:
+ * Missing, if any of the input arguments are missing.
+ * Null, if the arguments are null, if the list argument is not a list, or if the positional arguments are not numerical.
+ * Otherwise, it returns a new list, where the item at the old position is now in the new position, and all other
+ * items are moved accordingly.
+ */
+
+public class ArrayMoveDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private IAType inputListType;
+    private String funcIDString = String.valueOf(BuiltinFunctions.ARRAY_MOVE);
+
+    public static final IFunctionDescriptorFactory FACTORY =
+            DescriptorFactoryUtil.createFactory(ArrayMoveDescriptor::new, FunctionTypeInferers.SET_ARGUMENTS_TYPE);
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_MOVE;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        inputListType = (IAType) states[0];
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
+                return new ArrayMoveDescriptor.ArrayMoveEval(args, ctx);
+            }
+        };
+    }
+
+    public class ArrayMoveEval extends AbstractArrayMoveSwapEval {
+
+        private final ArrayBackedValueStorage storage;
+
+        ArrayMoveEval(IScalarEvaluatorFactory[] args, IEvaluatorContext ctx) throws HyracksDataException {
+            super(args, ctx, funcIDString, inputListType);
+            storage = new ArrayBackedValueStorage();
+        }
+
+        @Override
+        protected void buildList(int oldIndexInt, int newIndexInt, int listLen, ListAccessor listAccessor,
+                IAsterixListBuilder listBuilder) throws IOException {
+
+            for (int i = 0; i < listLen; i++) {
+
+                if (oldIndexInt < newIndexInt) {
+                    // if i outside of input indices range, just add items normally
+                    if (i < oldIndexInt || i > newIndexInt) {
+                        storage.reset();
+                        listAccessor.writeItem(i, storage.getDataOutput());
+                        listBuilder.addItem(storage);
+                    }
+                    // if within range, but not equal to the new index, then shift the items down by 1
+                    else if (i >= oldIndexInt && i < newIndexInt) {
+                        storage.reset();
+                        listAccessor.writeItem(i + 1, storage.getDataOutput());
+                        listBuilder.addItem(storage);
+                    }
+                    // if at new index position, then add the item that was at old index.
+                    else {
+                        storage.reset();
+                        listAccessor.writeItem(oldIndexInt, storage.getDataOutput());
+                        listBuilder.addItem(storage);
+                    }
+                } else {
+                    if (i < newIndexInt || i > oldIndexInt) {
+                        storage.reset();
+                        listAccessor.writeItem(i, storage.getDataOutput());
+                        listBuilder.addItem(storage);
+                    } else if (i > newIndexInt && i <= oldIndexInt) {
+                        storage.reset();
+                        listAccessor.writeItem(i - 1, storage.getDataOutput());
+                        listBuilder.addItem(storage);
+                    } else {
+                        storage.reset();
+                        listAccessor.writeItem(oldIndexInt, storage.getDataOutput());
+                        listBuilder.addItem(storage);
+                    }
+                }
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySwapDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySwapDescriptor.java
new file mode 100644
index 0000000..279982c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySwapDescriptor.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions;
+
+import java.io.IOException;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.asterix.runtime.utils.DescriptorFactoryUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+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.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+/**
+ * array_swap(list, position1, position2) returns a new list, switching the positions of both the items in position1
+ * and position2.
+ *
+ * It returns in order:
+ * Missing, if any of the input arguments are missing.
+ * Null, if the arguments are null, if the list argument is not a list, or if the positional arguments is not numerical.
+ * Otherwise, it returns a new list, where the two items at the
+ */
+
+public class ArraySwapDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private IAType inputListType;
+    private String funcIDString = String.valueOf(BuiltinFunctions.ARRAY_SWAP);
+
+    public static final IFunctionDescriptorFactory FACTORY =
+            DescriptorFactoryUtil.createFactory(ArraySwapDescriptor::new, FunctionTypeInferers.SET_ARGUMENTS_TYPE);
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_SWAP;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        inputListType = (IAType) states[0];
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
+                return new ArraySwapDescriptor.ArraySwapEval(args, ctx);
+            }
+        };
+    }
+
+    public class ArraySwapEval extends AbstractArrayMoveSwapEval {
+        private final ArrayBackedValueStorage storage;
+
+        ArraySwapEval(IScalarEvaluatorFactory[] args, IEvaluatorContext ctx) throws HyracksDataException {
+            super(args, ctx, funcIDString, inputListType);
+            storage = new ArrayBackedValueStorage();
+        }
+
+        @Override
+        protected void buildList(int oldIndexInt, int newIndexInt, int listLen, ListAccessor listAccessor,
+                IAsterixListBuilder listBuilder) throws IOException {
+            for (int i = 0; i < listLen; i++) {
+                if (oldIndexInt < newIndexInt) {
+                    if (i == oldIndexInt) {
+                        storage.reset();
+                        listAccessor.writeItem(newIndexInt, storage.getDataOutput());
+                        listBuilder.addItem(storage);
+                    }
+                    // the other case is when it is one of the two input indices, in which case we want to swap the two
+                    else if (i == newIndexInt) {
+                        storage.reset();
+                        listAccessor.writeItem(oldIndexInt, storage.getDataOutput());
+                        listBuilder.addItem(storage);
+                    } else {
+                        storage.reset();
+                        listAccessor.writeItem(i, storage.getDataOutput());
+                        listBuilder.addItem(storage);
+                    }
+                } else if (oldIndexInt > newIndexInt) {
+                    if (i == newIndexInt) {
+                        storage.reset();
+                        listAccessor.writeItem(oldIndexInt, storage.getDataOutput());
+                        listBuilder.addItem(storage);
+                    } else if (i == oldIndexInt) {
+                        storage.reset();
+                        listAccessor.writeItem(newIndexInt, storage.getDataOutput());
+                        listBuilder.addItem(storage);
+                    } else {
+                        storage.reset();
+                        listAccessor.writeItem(i, storage.getDataOutput());
+                        listBuilder.addItem(storage);
+                    }
+                }
+            }
+        }
+
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordAddPutEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordAddPutEvaluator.java
index 47e84eb..ed7da3f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordAddPutEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordAddPutEvaluator.java
@@ -16,101 +16,64 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.asterix.runtime.evaluators.functions.records;
 
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.om.pointables.ARecordVisitablePointable;
-import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.evaluators.functions.CastTypeEvaluator;
 import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.accessors.UTF8StringBinaryComparatorFactory;
 import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-abstract class AbstractRecordAddPutEvaluator implements IScalarEvaluator {
-
-    private final CastTypeEvaluator inputRecordCaster;
-    private final CastTypeEvaluator argRecordCaster;
+abstract class AbstractRecordAddPutEvaluator extends AbstractRecordFunctionEvaluator {
     private final IScalarEvaluator eval0;
     private final IScalarEvaluator eval1;
     private final IScalarEvaluator eval2;
-    final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
-    final DataOutput resultOutput = resultStorage.getDataOutput();
-    final IPointable inputRecordPointable = new VoidPointable();
-    final UTF8StringPointable newFieldNamePointable = new UTF8StringPointable();
-    final IPointable newFieldValuePointable = new VoidPointable();
-    final IBinaryComparator stringBinaryComparator =
-            UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
-    final RecordBuilder outRecordBuilder = new RecordBuilder();
-    final ARecordVisitablePointable inputOpenRecordPointable;
-    boolean newFieldValueIsMissing = false;
+    protected boolean newFieldValueIsMissing = false;
 
     AbstractRecordAddPutEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2,
-            IAType[] argTypes) {
+            ARecordType outRecType, ARecordType inRecType) {
+        super(outRecType, inRecType);
         this.eval0 = eval0;
         this.eval1 = eval1;
         this.eval2 = eval2;
-        inputOpenRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
-        inputRecordCaster = new CastTypeEvaluator(BuiltinType.ANY, argTypes[0], eval0);
-        argRecordCaster = new CastTypeEvaluator(BuiltinType.ANY, argTypes[2], eval2);
     }
 
     @Override
     public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
         resultStorage.reset();
-        eval0.evaluate(tuple, inputRecordPointable);
+        eval0.evaluate(tuple, inputPointable);
         eval1.evaluate(tuple, newFieldNamePointable);
         eval2.evaluate(tuple, newFieldValuePointable);
-        if (containsMissing(inputRecordPointable, newFieldNamePointable)) {
-            writeTypeTag(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
-            result.set(resultStorage);
+        ATypeTag inputTypeTag = PointableHelper.getTypeTag(inputPointable);
+        ATypeTag newFieldNameTypeTag = PointableHelper.getTypeTag(newFieldNamePointable);
+        if (inputTypeTag == ATypeTag.MISSING || newFieldNameTypeTag == ATypeTag.MISSING) {
+            PointableHelper.setMissing(result);
             return;
         }
-        final ATypeTag inputObjectType = PointableHelper.getTypeTag(inputRecordPointable);
-        final ATypeTag newFieldNameValueType = PointableHelper.getTypeTag(newFieldNamePointable);
-        if (inputObjectType != ATypeTag.OBJECT || newFieldNameValueType != ATypeTag.STRING) {
+        if (inputTypeTag != ATypeTag.OBJECT || newFieldNameTypeTag != ATypeTag.STRING) {
             PointableHelper.setNull(result);
             return;
         }
-        inputRecordCaster.evaluate(tuple, inputRecordPointable);
-        final ATypeTag newFieldValueTag = PointableHelper.getTypeTag(newFieldValuePointable);
-        if (newFieldValueTag.isDerivedType()) {
-            argRecordCaster.evaluate(tuple, newFieldValuePointable);
+        newFieldValueIsMissing = PointableHelper.getTypeTag(newFieldValuePointable) == ATypeTag.MISSING;
+        outputRecordTypeInfo.reset(outRecType);
+        if (inputRecordPointable == null) {
+            inputRecordPointable = pointableAllocator.allocateRecordValue(inRecType);
         }
-        newFieldValueIsMissing = newFieldValueTag == ATypeTag.MISSING;
-        buildOutputRecord();
+        buildOutputRecord(result);
         result.set(resultStorage);
     }
 
-    protected abstract void buildOutputRecord() throws HyracksDataException;
+    protected abstract void buildOutputRecord(IPointable result) throws HyracksDataException;
 
-    private boolean containsMissing(IPointable... pointables) {
-        for (int i = 0; i < pointables.length; i++) {
-            if (PointableHelper.getTypeTag(pointables[i]) == ATypeTag.MISSING) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    private void writeTypeTag(byte typeTag) throws HyracksDataException {
-        try {
-            resultOutput.writeByte(typeTag);
-        } catch (IOException e) {
-            throw HyracksDataException.create(e);
+    protected void addField(IPointable fieldName, IPointable fieldValue) throws HyracksDataException {
+        int pos = outputRecordTypeInfo.getFieldIndex(fieldName.getByteArray(), fieldName.getStartOffset() + 1,
+                fieldName.getLength() - 1);
+        if (pos >= 0) {
+            outRecordBuilder.addField(pos, fieldValue);
+        } else {
+            outRecordBuilder.addField(fieldName, fieldValue);
         }
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordFunctionEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordFunctionEvaluator.java
new file mode 100644
index 0000000..2bb192d
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordFunctionEvaluator.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.evaluators.functions.records;
+
+import java.io.DataOutput;
+
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.PointableAllocator;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.runtime.RuntimeRecordTypeInfo;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.data.std.accessors.UTF8StringBinaryComparatorFactory;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+/**
+ * Base evaluator class for the following functions:
+ * 1. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_ADD}
+ * 2. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_PUT}
+ * 3. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_REMOVE}
+ * 4. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_RENAME}
+ */
+abstract class AbstractRecordFunctionEvaluator implements IScalarEvaluator {
+    protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    protected final DataOutput resultOutput = resultStorage.getDataOutput();
+    protected final RecordBuilder outRecordBuilder = new RecordBuilder();
+
+    protected final IPointable newFieldNamePointable = new VoidPointable();
+    protected final IPointable newFieldValuePointable = new VoidPointable();
+    protected final IBinaryComparator stringBinaryComparator =
+            UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+
+    protected final PointableAllocator pointableAllocator = new PointableAllocator();
+    protected final IPointable inputPointable = new VoidPointable();
+    protected final ARecordType inRecType;
+    protected ARecordVisitablePointable inputRecordPointable;
+
+    protected final ARecordType outRecType;
+    protected final RuntimeRecordTypeInfo outputRecordTypeInfo = new RuntimeRecordTypeInfo();
+
+    AbstractRecordFunctionEvaluator(ARecordType outRecType, ARecordType inRecType) {
+        this.outRecType = outRecType;
+        this.inRecType = inRecType;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddDescriptor.java
index 67f2c8d..075b803 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddDescriptor.java
@@ -22,6 +22,8 @@
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.functions.FunctionTypeInferers;
@@ -32,7 +34,6 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class RecordAddDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
@@ -41,19 +42,18 @@
 
         @Override
         public IFunctionTypeInferer createFunctionTypeInferer() {
-            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+            return FunctionTypeInferers.RECORD_MODIFY_INFERER;
         }
     };
 
     private static final long serialVersionUID = 1L;
-    private IAType[] argTypes;
+    private ARecordType outRecType;
+    private ARecordType inRecType;
 
     @Override
     public void setImmutableStates(Object... states) {
-        argTypes = new IAType[states.length];
-        for (int i = 0; i < states.length; i++) {
-            argTypes[i] = (IAType) states[i];
-        }
+        outRecType = TypeComputeUtils.extractRecordType((IAType) states[0]);
+        inRecType = TypeComputeUtils.extractRecordType((IAType) states[1]);
     }
 
     @Override
@@ -67,7 +67,7 @@
                 for (int i = 0; i < args.length; i++) {
                     argEvals[i] = args[i].createScalarEvaluator(ctx);
                 }
-                return new RecordAddEvaluator(argEvals[0], argEvals[1], argEvals[2], argTypes);
+                return new RecordAddEvaluator(argEvals[0], argEvals[1], argEvals[2], outRecType, inRecType);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddEvaluator.java
index 52a69bf..3df16d0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddEvaluator.java
@@ -16,42 +16,41 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.asterix.runtime.evaluators.functions.records;
 
 import java.util.List;
 
-import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
 import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
 
 class RecordAddEvaluator extends AbstractRecordAddPutEvaluator {
-
-    RecordAddEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2, IAType[] argTypes) {
-        super(eval0, eval1, eval2, argTypes);
+    RecordAddEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2, ARecordType outRecType,
+            ARecordType inRecType) {
+        super(eval0, eval1, eval2, outRecType, inRecType);
     }
 
     @Override
-    protected void buildOutputRecord() throws HyracksDataException {
+    protected void buildOutputRecord(IPointable result) throws HyracksDataException {
         resultStorage.reset();
-        outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
-        outRecordBuilder.init();
-        inputOpenRecordPointable.set(inputRecordPointable);
-        final List<IVisitablePointable> fieldNames = inputOpenRecordPointable.getFieldNames();
-        final List<IVisitablePointable> fieldValues = inputOpenRecordPointable.getFieldValues();
+        outRecordBuilder.reset(outRecType);
+        inputRecordPointable.set(inputPointable);
+        final List<IVisitablePointable> fieldNames = inputRecordPointable.getFieldNames();
+        final List<IVisitablePointable> fieldValues = inputRecordPointable.getFieldValues();
         boolean newFieldFound = false;
         for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
             final IVisitablePointable fieldName = fieldNames.get(i);
-            if (PointableHelper.isEqual(fieldName, newFieldNamePointable, stringBinaryComparator)) {
+            final IVisitablePointable fieldValue = fieldValues.get(i);
+            if (!newFieldFound && PointableHelper.isEqual(fieldName, newFieldNamePointable, stringBinaryComparator)) {
                 newFieldFound = true;
             }
-            outRecordBuilder.addField(fieldName, fieldValues.get(i));
+            addField(fieldName, fieldValue);
         }
         if (!newFieldValueIsMissing && !newFieldFound) {
-            outRecordBuilder.addField(newFieldNamePointable, newFieldValuePointable);
+            addField(newFieldNamePointable, newFieldValuePointable);
         }
         outRecordBuilder.write(resultOutput, true);
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutDescriptor.java
index e4d72e5..870a2e8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutDescriptor.java
@@ -22,6 +22,8 @@
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.functions.FunctionTypeInferers;
@@ -32,7 +34,6 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class RecordPutDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
@@ -41,19 +42,18 @@
 
         @Override
         public IFunctionTypeInferer createFunctionTypeInferer() {
-            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+            return FunctionTypeInferers.RECORD_MODIFY_INFERER;
         }
     };
 
     private static final long serialVersionUID = 1L;
-    private IAType[] argTypes;
+    private ARecordType outRecType;
+    private ARecordType inRecType;
 
     @Override
     public void setImmutableStates(Object... states) {
-        argTypes = new IAType[states.length];
-        for (int i = 0; i < states.length; i++) {
-            argTypes[i] = (IAType) states[i];
-        }
+        outRecType = TypeComputeUtils.extractRecordType((IAType) states[0]);
+        inRecType = TypeComputeUtils.extractRecordType((IAType) states[1]);
     }
 
     @Override
@@ -67,7 +67,7 @@
                 for (int i = 0; i < args.length; i++) {
                     argEvals[i] = args[i].createScalarEvaluator(ctx);
                 }
-                return new RecordPutEvaluator(argEvals[0], argEvals[1], argEvals[2], argTypes);
+                return new RecordPutEvaluator(argEvals[0], argEvals[1], argEvals[2], outRecType, inRecType);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutEvaluator.java
index 857bd2e..3a9f3f8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutEvaluator.java
@@ -16,51 +16,46 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.asterix.runtime.evaluators.functions.records;
 
 import java.util.List;
 
-import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
 import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
 
 class RecordPutEvaluator extends AbstractRecordAddPutEvaluator {
-
-    RecordPutEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2, IAType[] argTypes) {
-        super(eval0, eval1, eval2, argTypes);
+    RecordPutEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2, ARecordType outRecType,
+            ARecordType inRecType) {
+        super(eval0, eval1, eval2, outRecType, inRecType);
     }
 
     @Override
-    protected void buildOutputRecord() throws HyracksDataException {
+    protected void buildOutputRecord(IPointable result) throws HyracksDataException {
         resultStorage.reset();
-        outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
-        outRecordBuilder.init();
-        inputOpenRecordPointable.set(inputRecordPointable);
-        final List<IVisitablePointable> fieldNames = inputOpenRecordPointable.getFieldNames();
-        final List<IVisitablePointable> fieldValues = inputOpenRecordPointable.getFieldValues();
+        outRecordBuilder.reset(outRecType);
+        inputRecordPointable.set(inputPointable);
+        final List<IVisitablePointable> fieldNames = inputRecordPointable.getFieldNames();
+        final List<IVisitablePointable> fieldValues = inputRecordPointable.getFieldValues();
         boolean newFieldFound = false;
         for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
             final IVisitablePointable fieldName = fieldNames.get(i);
-            if (!PointableHelper.isEqual(fieldName, newFieldNamePointable, stringBinaryComparator)) {
-                outRecordBuilder.addField(fieldName, fieldValues.get(i));
+            final IVisitablePointable fieldValue = fieldValues.get(i);
+            if (!newFieldFound && !PointableHelper.isEqual(fieldName, newFieldNamePointable, stringBinaryComparator)) {
+                addField(fieldName, fieldValue);
             } else {
                 newFieldFound = true;
                 if (!newFieldValueIsMissing) {
-                    putNewField();
+                    addField(newFieldNamePointable, newFieldValuePointable);
                 }
             }
         }
         if (!newFieldFound) {
-            putNewField();
+            addField(newFieldNamePointable, newFieldValuePointable);
         }
         outRecordBuilder.write(resultOutput, true);
     }
-
-    private void putNewField() throws HyracksDataException {
-        outRecordBuilder.addField(newFieldNamePointable, newFieldValuePointable);
-    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java
index 8e44ada..818b650 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java
@@ -23,7 +23,9 @@
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
 import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,7 +36,6 @@
 
 @MissingNullInOutFunction
 public class RecordRemoveDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
@@ -43,16 +44,18 @@
 
         @Override
         public IFunctionTypeInferer createFunctionTypeInferer() {
-            return FunctionTypeInferers.RecordAccessorTypeInferer.INSTANCE_LAX;
+            return FunctionTypeInferers.RECORD_MODIFY_INFERER;
         }
     };
 
     private static final long serialVersionUID = 1L;
-    private ARecordType recordType;
+    private ARecordType outRecType;
+    private ARecordType inRecType;
 
     @Override
     public void setImmutableStates(Object... states) {
-        recordType = (ARecordType) states[0];
+        outRecType = TypeComputeUtils.extractRecordType((IAType) states[0]);
+        inRecType = TypeComputeUtils.extractRecordType((IAType) states[1]);
     }
 
     @Override
@@ -66,7 +69,7 @@
                 for (int i = 0; i < args.length; i++) {
                     argEvals[i] = args[i].createScalarEvaluator(ctx);
                 }
-                return new RecordRemoveEvaluator(argEvals[0], argEvals[1], recordType);
+                return new RecordRemoveEvaluator(argEvals[0], argEvals[1], outRecType, inRecType);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java
index 51f66b0..e560c10 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java
@@ -16,140 +16,87 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.asterix.runtime.evaluators.functions.records;
 
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.om.pointables.ARecordVisitablePointable;
-import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
 import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.pointables.cast.ACastVisitor;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.accessors.UTF8StringBinaryComparatorFactory;
 import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-class RecordRemoveEvaluator implements IScalarEvaluator {
-
-    private final IPointable inputRecordPointable = new VoidPointable();
-    private final UTF8StringPointable fieldToRemovePointable = new UTF8StringPointable();
-    private final IBinaryComparator stringBinaryComparator =
-            UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
-    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
-    private final DataOutput resultOutput = resultStorage.getDataOutput();
-    private final RecordBuilder outRecordBuilder = new RecordBuilder();
+class RecordRemoveEvaluator extends AbstractRecordFunctionEvaluator {
     private final IScalarEvaluator eval0;
     private final IScalarEvaluator eval1;
-    private final ARecordVisitablePointable openRecordPointable;
-    private ARecordVisitablePointable inputRecordVisitable;
-    private boolean requiresCast = false;
-    private ACastVisitor castVisitor;
-    private Triple<IVisitablePointable, IAType, Boolean> castVisitorArg;
+    private final IPointable fieldToRemovePointable = new VoidPointable();
 
-    RecordRemoveEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, ARecordType recordType) {
+    RecordRemoveEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, ARecordType outRecType,
+            ARecordType inRecType) {
+        super(outRecType, inRecType);
         this.eval0 = eval0;
         this.eval1 = eval1;
-        openRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
-        if (recordType != null) {
-            inputRecordVisitable = new ARecordVisitablePointable(recordType);
-            if (hasDerivedType(recordType.getFieldTypes())) {
-                requiresCast = true;
-                castVisitor = new ACastVisitor();
-                castVisitorArg =
-                        new Triple<>(openRecordPointable, openRecordPointable.getInputRecordType(), Boolean.FALSE);
-            }
-        }
     }
 
     @Override
     public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
         resultStorage.reset();
-        boolean returnNull = false;
-        eval0.evaluate(tuple, inputRecordPointable);
+        eval0.evaluate(tuple, inputPointable);
         eval1.evaluate(tuple, fieldToRemovePointable);
-
-        if (PointableHelper.checkAndSetMissingOrNull(result, inputRecordPointable, fieldToRemovePointable)) {
+        if (PointableHelper.checkAndSetMissingOrNull(result, inputPointable, fieldToRemovePointable)) {
             return;
         }
 
-        byte[] data = inputRecordPointable.getByteArray();
-        int offset = inputRecordPointable.getStartOffset();
+        // Check the type of our first argument.
+        byte[] data = inputPointable.getByteArray();
+        int offset = inputPointable.getStartOffset();
         byte typeTag = data[offset];
         if (typeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-            returnNull = true;
+            PointableHelper.setNull(result);
+            return;
         }
 
+        // Check the type of our second argument.
         data = fieldToRemovePointable.getByteArray();
         offset = fieldToRemovePointable.getStartOffset();
         typeTag = data[offset];
         if (typeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            returnNull = true;
-        }
-        if (returnNull) {
             PointableHelper.setNull(result);
             return;
         }
-        evaluate();
-        result.set(resultStorage);
-    }
 
-    private void evaluate() throws HyracksDataException {
-        resultStorage.reset();
         try {
-            final ARecordVisitablePointable inputRecord = getInputRecordVisitablePointable();
-            buildOutputRecord(inputRecord);
+            outRecordBuilder.reset(outRecType);
+            outputRecordTypeInfo.reset(outRecType);
+            if (inputRecordPointable == null) {
+                inputRecordPointable = pointableAllocator.allocateRecordValue(inRecType);
+            }
+            inputRecordPointable.set(inputPointable);
+            final List<IVisitablePointable> fieldNames = inputRecordPointable.getFieldNames();
+            final List<IVisitablePointable> fieldValues = inputRecordPointable.getFieldValues();
+            for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
+                final IVisitablePointable fieldName = fieldNames.get(i);
+                final IVisitablePointable fieldValue = fieldValues.get(i);
+                if (!PointableHelper.isEqual(fieldName, fieldToRemovePointable, stringBinaryComparator)) {
+                    int pos = outputRecordTypeInfo.getFieldIndex(fieldName.getByteArray(),
+                            fieldName.getStartOffset() + 1, fieldName.getLength() - 1);
+                    if (pos >= 0) {
+                        outRecordBuilder.addField(pos, fieldValue);
+                    } else {
+                        outRecordBuilder.addField(fieldName, fieldValue);
+                    }
+                }
+            }
+            outRecordBuilder.write(resultOutput, true);
+
         } catch (IOException e) {
             throw HyracksDataException.create(e);
         }
-    }
-
-    private void buildOutputRecord(ARecordVisitablePointable inputRecord) throws HyracksDataException {
-        outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
-        outRecordBuilder.init();
-        final List<IVisitablePointable> fieldNames = inputRecord.getFieldNames();
-        final List<IVisitablePointable> fieldValues = inputRecord.getFieldValues();
-        for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
-            final IVisitablePointable fieldName = fieldNames.get(i);
-            if (!PointableHelper.isEqual(fieldName, fieldToRemovePointable, stringBinaryComparator)) {
-                outRecordBuilder.addField(fieldName, fieldValues.get(i));
-            }
-        }
-        outRecordBuilder.write(resultOutput, true);
-    }
-
-    private ARecordVisitablePointable getInputRecordVisitablePointable() throws HyracksDataException {
-        inputRecordVisitable.set(inputRecordPointable);
-        if (requiresCast) {
-            return castToOpenRecord();
-        }
-        return inputRecordVisitable;
-    }
-
-    private boolean hasDerivedType(IAType[] types) {
-        for (IAType type : types) {
-            if (type.getTypeTag().isDerivedType()) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    private ARecordVisitablePointable castToOpenRecord() throws HyracksDataException {
-        inputRecordVisitable.accept(castVisitor, castVisitorArg);
-        return openRecordPointable;
+        result.set(resultStorage);
     }
 }
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/evaluators/functions/records/RecordRenameDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameDescriptor.java
index 8b345aa..ea4f6db 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameDescriptor.java
@@ -23,7 +23,9 @@
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
 import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,7 +36,6 @@
 
 @MissingNullInOutFunction
 public class RecordRenameDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
@@ -43,16 +44,18 @@
 
         @Override
         public IFunctionTypeInferer createFunctionTypeInferer() {
-            return FunctionTypeInferers.RecordAccessorTypeInferer.INSTANCE_LAX;
+            return FunctionTypeInferers.RECORD_MODIFY_INFERER;
         }
     };
 
     private static final long serialVersionUID = 1L;
-    private ARecordType recordType;
+    private ARecordType outRecType;
+    private ARecordType inRecType;
 
     @Override
     public void setImmutableStates(Object... states) {
-        recordType = (ARecordType) states[0];
+        outRecType = TypeComputeUtils.extractRecordType((IAType) states[0]);
+        inRecType = TypeComputeUtils.extractRecordType((IAType) states[1]);
     }
 
     @Override
@@ -66,7 +69,7 @@
                 for (int i = 0; i < args.length; i++) {
                     argEvals[i] = args[i].createScalarEvaluator(ctx);
                 }
-                return new RecordRenameEvaluator(argEvals[0], argEvals[1], argEvals[2], recordType);
+                return new RecordRenameEvaluator(argEvals[0], argEvals[1], argEvals[2], outRecType, inRecType);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java
index ac09ffa..1461536 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java
@@ -16,156 +16,105 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.asterix.runtime.evaluators.functions.records;
 
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.om.pointables.ARecordVisitablePointable;
-import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
 import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.pointables.cast.ACastVisitor;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.accessors.UTF8StringBinaryComparatorFactory;
 import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-class RecordRenameEvaluator implements IScalarEvaluator {
-
-    private final IPointable inputRecordPointable = new VoidPointable();
-    private final UTF8StringPointable oldFieldNamePointable = new UTF8StringPointable();
-    private final UTF8StringPointable newFieldNamePointable = new UTF8StringPointable();
-    private final IBinaryComparator stringBinaryComparator =
-            UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
-    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
-    private final DataOutput resultOutput = resultStorage.getDataOutput();
-    private final RecordBuilder outRecordBuilder = new RecordBuilder();
+class RecordRenameEvaluator extends AbstractRecordFunctionEvaluator {
     private final IScalarEvaluator eval0;
     private final IScalarEvaluator eval1;
     private final IScalarEvaluator eval2;
-    private final ARecordVisitablePointable openRecordPointable;
-    private ARecordVisitablePointable inputRecordVisitable;
-    private boolean requiresCast = false;
-    private ACastVisitor castVisitor;
-    private Triple<IVisitablePointable, IAType, Boolean> castVisitorArg;
+    private final IPointable oldFieldNamePointable = new VoidPointable();
 
     RecordRenameEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2,
-            ARecordType recordType) {
+            ARecordType outRecType, ARecordType inRecType) {
+        super(outRecType, inRecType);
         this.eval0 = eval0;
         this.eval1 = eval1;
         this.eval2 = eval2;
-        openRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
-        if (recordType != null) {
-            inputRecordVisitable = new ARecordVisitablePointable(recordType);
-            if (hasDerivedType(recordType.getFieldTypes())) {
-                requiresCast = true;
-                castVisitor = new ACastVisitor();
-                castVisitorArg =
-                        new Triple<>(openRecordPointable, openRecordPointable.getInputRecordType(), Boolean.FALSE);
-            }
-        }
     }
 
     @Override
     public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
         resultStorage.reset();
-        boolean returnNull = false;
-        eval0.evaluate(tuple, inputRecordPointable);
+        eval0.evaluate(tuple, inputPointable);
         eval1.evaluate(tuple, oldFieldNamePointable);
         eval2.evaluate(tuple, newFieldNamePointable);
-
-        if (PointableHelper.checkAndSetMissingOrNull(result, inputRecordPointable, oldFieldNamePointable,
+        if (PointableHelper.checkAndSetMissingOrNull(result, inputPointable, oldFieldNamePointable,
                 newFieldNamePointable)) {
             return;
         }
 
-        byte[] data = inputRecordPointable.getByteArray();
-        int offset = inputRecordPointable.getStartOffset();
+        // Check the type of our first argument.
+        byte[] data = inputPointable.getByteArray();
+        int offset = inputPointable.getStartOffset();
         byte typeTag = data[offset];
         if (typeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-            returnNull = true;
+            PointableHelper.setNull(result);
+            return;
         }
 
+        // Check the type of our second argument.
         data = oldFieldNamePointable.getByteArray();
         offset = oldFieldNamePointable.getStartOffset();
         typeTag = data[offset];
         if (typeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            returnNull = true;
+            PointableHelper.setNull(result);
+            return;
         }
 
+        // Check the type of our third argument.
         data = newFieldNamePointable.getByteArray();
         offset = newFieldNamePointable.getStartOffset();
         typeTag = data[offset];
         if (typeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            returnNull = true;
-        }
-        if (returnNull) {
             PointableHelper.setNull(result);
             return;
         }
-        evaluate();
-        result.set(resultStorage);
-    }
 
-    private void evaluate() throws HyracksDataException {
-        resultStorage.reset();
         try {
-            final ARecordVisitablePointable inputRecord = getInputRecordVisitablePointable();
-            buildOutputRecord(inputRecord);
+            outRecordBuilder.reset(outRecType);
+            outputRecordTypeInfo.reset(outRecType);
+            if (inputRecordPointable == null) {
+                inputRecordPointable = pointableAllocator.allocateRecordValue(inRecType);
+            }
+            inputRecordPointable.set(inputPointable);
+            final List<IVisitablePointable> fieldNames = inputRecordPointable.getFieldNames();
+            final List<IVisitablePointable> fieldValues = inputRecordPointable.getFieldValues();
+            for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
+                final IVisitablePointable fieldName = fieldNames.get(i);
+                final IVisitablePointable fieldValue = fieldValues.get(i);
+                if (!PointableHelper.isEqual(fieldName, oldFieldNamePointable, stringBinaryComparator)) {
+                    addField(fieldName, fieldValue);
+                } else {
+                    addField(newFieldNamePointable, fieldValue);
+                }
+            }
+            outRecordBuilder.write(resultOutput, true);
         } catch (IOException e) {
             throw HyracksDataException.create(e);
         }
+        result.set(resultStorage);
     }
 
-    private void buildOutputRecord(ARecordVisitablePointable inputRecord) throws HyracksDataException {
-        outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
-        outRecordBuilder.init();
-        final List<IVisitablePointable> fieldNames = inputRecord.getFieldNames();
-        final List<IVisitablePointable> fieldValues = inputRecord.getFieldValues();
-        for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
-            final IVisitablePointable fieldName = fieldNames.get(i);
-            if (!PointableHelper.isEqual(fieldName, oldFieldNamePointable, stringBinaryComparator)) {
-                outRecordBuilder.addField(fieldName, fieldValues.get(i));
-            } else {
-                outRecordBuilder.addField(newFieldNamePointable, fieldValues.get(i));
-            }
+    private void addField(IPointable fieldName, IPointable fieldValue) throws HyracksDataException {
+        int pos = outputRecordTypeInfo.getFieldIndex(fieldName.getByteArray(), fieldName.getStartOffset() + 1,
+                fieldName.getLength() - 1);
+        if (pos >= 0) {
+            outRecordBuilder.addField(pos, fieldValue);
+        } else {
+            outRecordBuilder.addField(fieldName, fieldValue);
         }
-        outRecordBuilder.write(resultOutput, true);
     }
-
-    private ARecordVisitablePointable getInputRecordVisitablePointable() throws HyracksDataException {
-        inputRecordVisitable.set(inputRecordPointable);
-        if (requiresCast) {
-            return castToOpenRecord();
-        }
-        return inputRecordVisitable;
-    }
-
-    private boolean hasDerivedType(IAType[] types) {
-        for (IAType type : types) {
-            if (type.getTypeTag().isDerivedType()) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    private ARecordVisitablePointable castToOpenRecord() throws HyracksDataException {
-        inputRecordVisitable.accept(castVisitor, castVisitorArg);
-        return openRecordPointable;
-    }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index 4b418b6..63dae91 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -327,6 +327,7 @@
 import org.apache.asterix.runtime.evaluators.functions.AndDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.AnyCollectionMemberDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayAppendDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayBinarySearchDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayConcatDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayContainsDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayDistinctDescriptor;
@@ -335,6 +336,7 @@
 import org.apache.asterix.runtime.evaluators.functions.ArrayIfNullDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayInsertDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayIntersectDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayMoveDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayPositionDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayPrependDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayPutDescriptor;
@@ -349,6 +351,7 @@
 import org.apache.asterix.runtime.evaluators.functions.ArraySliceWithoutEndPositionDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArraySortDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayStarDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArraySwapDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArraySymDiffDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArraySymDiffnDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayUnionDescriptor;
@@ -671,6 +674,9 @@
         fc.add(ArraySymDiffnDescriptor.FACTORY);
         fc.add(ArrayStarDescriptor.FACTORY);
         fc.add(ArrayExceptDescriptor.FACTORY);
+        fc.add(ArrayMoveDescriptor.FACTORY);
+        fc.add(ArraySwapDescriptor.FACTORY);
+        fc.add(ArrayBinarySearchDescriptor.FACTORY);
 
         // unnesting functions
         fc.add(TidRunningAggregateDescriptor.FACTORY);
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..5abebd2 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
@@ -116,6 +116,16 @@
         }
     };
 
+    public static final IFunctionTypeInferer RECORD_MODIFY_INFERER = (expr, fd, context, compilerProps) -> {
+        AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
+        IAType outType = (IAType) context.getType(expr);
+        IAType inType = (IAType) context.getType(f.getArguments().get(0).getValue());
+        if (inType.getTypeTag().equals(ATypeTag.ANY)) {
+            inType = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+        }
+        fd.setImmutableStates(outType, inType);
+    };
+
     public static final class CastTypeInferer implements IFunctionTypeInferer {
         @Override
         public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
@@ -300,6 +310,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/DatasetStreamStats.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/DatasetStreamStats.java
index 8ea267a..16135ee 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/DatasetStreamStats.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/DatasetStreamStats.java
@@ -37,14 +37,14 @@
 
     public DatasetStreamStats(IOperatorStats opStats) {
         this.cardinality = opStats.getTupleCounter().get();
-        long totalTupleSize = opStats.getDiskIoCounter().get();
+        long totalTupleSize = opStats.getPageReads().get();
         this.avgTupleSize = cardinality > 0 ? (int) (totalTupleSize / cardinality) : 0;
         this.indexesStats = opStats.getIndexesStats();
     }
 
     static void update(IOperatorStats opStats, long tupleCount, long tupleSize, Map<String, IndexStats> indexStats) {
         opStats.getTupleCounter().update(tupleCount);
-        opStats.getDiskIoCounter().update(tupleSize);
+        opStats.getPageReads().update(tupleSize);
         opStats.updateIndexesStats(indexStats);
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/DatasetStreamStatsOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/DatasetStreamStatsOperatorDescriptor.java
index 4f2e419..f9d75b5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/DatasetStreamStatsOperatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/DatasetStreamStatsOperatorDescriptor.java
@@ -19,6 +19,8 @@
 
 package org.apache.asterix.runtime.operators;
 
+import static org.apache.hyracks.api.job.profiling.NoOpOperatorStats.INVALID_ODID;
+
 import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.Map;
@@ -83,7 +85,7 @@
                 writer.open();
                 IStatsCollector coll = ctx.getStatsCollector();
                 if (coll != null) {
-                    coll.add(new OperatorStats(operatorName));
+                    coll.add(new OperatorStats(operatorName, INVALID_ODID));
                 }
                 INCServiceContext serviceCtx = ctx.getJobletContext().getServiceContext();
                 indexStats = new HashMap<>();
@@ -134,7 +136,7 @@
             public void close() throws HyracksDataException {
                 IStatsCollector statsCollector = ctx.getStatsCollector();
                 if (statsCollector != null) {
-                    IOperatorStats stats = statsCollector.getOrAddOperatorStats(operatorName);
+                    IOperatorStats stats = statsCollector.getOperatorStats(operatorName);
                     DatasetStreamStats.update(stats, totalTupleCount, totalTupleLength, indexStats);
                 }
                 writer.close();
diff --git a/asterixdb/asterix-server/pom.xml b/asterixdb/asterix-server/pom.xml
index 344f915..0fcb828 100644
--- a/asterixdb/asterix-server/pom.xml
+++ b/asterixdb/asterix-server/pom.xml
@@ -352,6 +352,12 @@
             </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>
@@ -420,6 +426,23 @@
             </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>
+              <gavs>
                 <gav>io.grpc:grpc-alts:1.52.1</gav>
                 <gav>io.grpc:grpc-api:1.52.1</gav>
                 <gav>io.grpc:grpc-auth:1.52.1</gav>
@@ -462,12 +485,67 @@
               <url>https://raw.githubusercontent.com/perfmark/perfmark/v0.26.0/LICENSE</url>
               <noticeUrl>https://raw.githubusercontent.com/perfmark/perfmark/v0.26.0/NOTICE</noticeUrl>
             </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>
           </overrides>
           <licenses>
             <license>
                 <displayName>Various 3rd party</displayName>
-                <url>file://${basedir}}/../asterix-dashboard/src/main/resources/dashboard/static/3rdpartylicenses.txt</url>
-                <contentFile>${basedir}}/../asterix-dashboard/src/main/resources/dashboard/static/3rdpartylicenses.txt</contentFile>
+                <url>file://${basedir}/../asterix-dashboard/src/main/resources/dashboard/static/3rdpartylicenses.txt</url>
+                <contentFile>${basedir}/../asterix-dashboard/src/main/resources/dashboard/static/3rdpartylicenses.txt</contentFile>
             </license>
             <license>
               <displayName>a BSD 3-clause license</displayName>
@@ -513,6 +591,18 @@
                 <aliasUrl>https://raw.githubusercontent.com/googleapis/googleapis/master/LICENSE</aliasUrl>
                 <aliasUrl>https://raw.githubusercontent.com/perfmark/perfmark/v0.26.0/LICENSE</aliasUrl>
                 <aliasUrl>https://raw.githubusercontent.com/googleapis/google-http-java-client/v1.42.3/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/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.9.4/LICENSE</aliasUrl>
                 <aliasUrl>https://raw.githubusercontent.com/google/gson/gson-parent-2.10.1/LICENSE</aliasUrl>
@@ -520,6 +610,8 @@
                 <aliasUrl>https://raw.githubusercontent.com/googleapis/google-api-java-client/v2.1.2/LICENSE</aliasUrl>
                 <aliasUrl>https://raw.githubusercontent.com/grpc/grpc-java/v1.52.1/LICENSE</aliasUrl>
                 <aliasUrl>https://raw.githubusercontent.com/reactor/reactor-netty/v1.0.28/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>
               </aliasUrls>
               <metric>1</metric>
             </license>
@@ -922,7 +1014,7 @@
           <plugin>
             <artifactId>jdeb</artifactId>
             <groupId>org.vafer</groupId>
-            <version>1.5</version>
+            <version>1.8</version>
             <executions>
               <execution>
                 <phase>package</phase>
@@ -932,26 +1024,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>
@@ -974,6 +1076,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>
@@ -1019,7 +1154,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
-      <artifactId>hyracks-server</artifactId>
+      <artifactId>hyracks-server-test</artifactId>
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>
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/pom.xml b/asterixdb/pom.xml
index d1b7faf..ff4f38a 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -46,8 +46,7 @@
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <file.encoding>UTF-8</file.encoding>
     <jvm.extraargs />
-    <source.jdk.version>11</source.jdk.version>
-    <target.jdk.version>11</target.jdk.version>
+    <maven.compiler.release>11</maven.compiler.release>
     <javac.xlint.value>all</javac.xlint.value>
     <source-format.goal>format</source-format.goal>
     <source-format.skip>false</source-format.skip>
@@ -62,6 +61,8 @@
     <skip.surefire.tests>${skipTests}</skip.surefire.tests>
     <skip.testResources>${maven.test.skip}</skip.testResources>
     <testLog4jConfigFile>${root.dir}/asterix-app/src/test/resources/log4j2-asterixdb-test.xml</testLog4jConfigFile>
+    <extraSurefireArgLine/>
+    <extraFailsafeArgLine/>
 
     <!-- Definition of tests in various categories which may be excluded -->
     <repeated.tests>**/RepeatedTest.java</repeated.tests>
@@ -86,16 +87,17 @@
     <hyracks.version>0.3.8.2-SNAPSHOT</hyracks.version>
     <hadoop.version>3.3.6</hadoop.version>
     <jacoco.version>0.7.6.201602180812</jacoco.version>
-    <log4j.version>2.19.0</log4j.version>
+    <log4j.version>2.22.1</log4j.version>
     <awsjavasdk.version>2.23.3</awsjavasdk.version>
     <parquet.version>1.12.3</parquet.version>
-    <hadoop-awsjavasdk.version>1.12.402</hadoop-awsjavasdk.version>
+    <hadoop-awsjavasdk.version>1.12.637</hadoop-awsjavasdk.version>
     <azureblobjavasdk.version>12.25.1</azureblobjavasdk.version>
     <azurecommonjavasdk.version>12.24.1</azurecommonjavasdk.version>
     <azureidentity.version>1.11.1</azureidentity.version>
     <azuredatalakejavasdk.version>12.18.1</azuredatalakejavasdk.version>
     <gcsjavasdk.version>2.26.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>
@@ -145,6 +147,7 @@
             -Xdebug
             -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=${debug.suspend.flag}
             ${coverageArgLine}
+            ${extraSurefireArgLine}
           </argLine>
           <includes>
             <include>${test.includes}</include>
@@ -162,7 +165,10 @@
           <runOrder>alphabetical</runOrder>
           <forkCount>1</forkCount>
           <reuseForks>false</reuseForks>
-          <argLine>${coverageArgLine}</argLine>
+          <argLine>
+            ${coverageArgLine}
+            ${extraFailsafeArgLine}
+          </argLine>
           <systemProperties>
             <log4j.configurationFile>${testLog4jConfigFile}</log4j.configurationFile>
           </systemProperties>
@@ -219,8 +225,6 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-compiler-plugin</artifactId>
         <configuration>
-          <source>${source.jdk.version}</source>
-          <target>${target.jdk.version}</target>
           <compilerArgument>-Xlint:${javac.xlint.value}</compilerArgument>
         </configuration>
       </plugin>
@@ -246,7 +250,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
-        <version>3.0.0</version>
+        <version>3.3.1</version>
         <executions>
           <execution>
             <id>verify-style</id>
@@ -331,7 +335,7 @@
                   <version>[3.3.9,)</version>
                 </requireMavenVersion>
                 <requireJavaVersion>
-                  <version>[11,)</version>
+                  <version>[${maven.compiler.release},)</version>
                 </requireJavaVersion>
               </rules>
             </configuration>
@@ -598,7 +602,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-dependency-plugin</artifactId>
-          <version>3.1.2</version>
+          <version>3.6.0</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
@@ -608,7 +612,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-plugin-plugin</artifactId>
-          <version>3.6.0</version>
+          <version>3.9.0</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
@@ -939,7 +943,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>
@@ -964,7 +968,7 @@
       <dependency>
         <groupId>org.slf4j</groupId>
         <artifactId>slf4j-simple</artifactId>
-        <version>1.7.33</version>
+        <version>1.7.36</version>
       </dependency>
       <dependency>
         <groupId>org.apache.maven</groupId>
@@ -1204,7 +1208,7 @@
       <dependency>
         <groupId>org.apache.commons</groupId>
         <artifactId>commons-compress</artifactId>
-        <version>1.24.0</version>
+        <version>1.25.0</version>
       </dependency>
       <dependency>
         <groupId>commons-logging</groupId>
@@ -1214,7 +1218,7 @@
       <dependency>
         <groupId>commons-logging</groupId>
         <artifactId>commons-logging</artifactId>
-        <version>1.2</version>
+        <version>1.3.0</version>
       </dependency>
       <dependency>
         <groupId>org.apache.hyracks</groupId>
@@ -1314,7 +1318,7 @@
       </dependency>
       <dependency>
         <groupId>org.apache.hyracks</groupId>
-        <artifactId>hyracks-server</artifactId>
+        <artifactId>hyracks-server-test</artifactId>
         <version>${hyracks.version}</version>
         <type>test-jar</type>
       </dependency>
@@ -1454,7 +1458,7 @@
       <dependency>
         <groupId>commons-codec</groupId>
         <artifactId>commons-codec</artifactId>
-        <version>1.15</version>
+        <version>1.16.0</version>
       </dependency>
       <dependency>
         <groupId>it.unimi.dsi</groupId>
@@ -1830,7 +1834,7 @@
       <dependency>
         <groupId>org.apache.commons</groupId>
         <artifactId>commons-csv</artifactId>
-        <version>1.9.0</version>
+        <version>1.10.0</version>
       </dependency>
       <dependency>
         <groupId>org.testcontainers</groupId>
@@ -1845,7 +1849,7 @@
       <dependency>
         <groupId>org.apache.httpcomponents</groupId>
         <artifactId>httpmime</artifactId>
-        <version>4.5.11</version>
+        <version>4.5.14</version>
       </dependency>
       <dependency>
         <groupId>org.msgpack</groupId>
@@ -1934,11 +1938,38 @@
         <version>${hadoop.version}</version>
       </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>
+          <!-- exclude transitive grpc-api,grpc-census dependencies, as they are bringing in older versions -->
+          <exclusion>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-api</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>io.grpc</groupId>
+            <artifactId>grpc-census</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <!-- Hadoop GCS end -->
       <dependency>
         <groupId>org.codehaus.jettison</groupId>
         <artifactId>jettison</artifactId>
         <version>1.5.4</version>
       </dependency>
+      <dependency>
+        <groupId>io.grpc</groupId>
+        <artifactId>grpc-census</artifactId>
+        <version>1.56.1</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
 
diff --git a/asterixdb/src/main/appended-resources/supplemental-models.xml b/asterixdb/src/main/appended-resources/supplemental-models.xml
index b586d9a..e6f0362 100644
--- a/asterixdb/src/main/appended-resources/supplemental-models.xml
+++ b/asterixdb/src/main/appended-resources/supplemental-models.xml
@@ -37,13 +37,13 @@
       <licenses>
         <license>
           <name>an MIT-style license</name>
-          <url>https://raw.githubusercontent.com/qos-ch/slf4j/v_1.7.25/slf4j-api/LICENSE.txt</url>
+          <url>https://raw.githubusercontent.com/qos-ch/slf4j/v_1.7.36/slf4j-api/LICENSE.txt</url>
         </license>
       </licenses>
       <properties>
-        <license.ignoreMissingEmbeddedNotice>1.7.25</license.ignoreMissingEmbeddedNotice>
-        <license.ignoreMissingEmbeddedLicense>1.7.25</license.ignoreMissingEmbeddedLicense>
-        <license.ignoreLicenseOverride>1.7.25</license.ignoreLicenseOverride>
+        <license.ignoreMissingEmbeddedNotice>1.7.36</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreMissingEmbeddedLicense>1.7.36</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreLicenseOverride>1.7.36</license.ignoreLicenseOverride>
       </properties>
     </project>
   </supplement>
@@ -511,8 +511,8 @@
       <groupId>com.amazonaws</groupId>
       <artifactId>aws-java-sdk-core</artifactId>
       <properties>
-        <license.ignoreMissingEmbeddedLicense>1.12.402</license.ignoreMissingEmbeddedLicense>
-        <license.ignoreMissingEmbeddedNotice>1.12.402</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreMissingEmbeddedLicense>1.12.637</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.12.637</license.ignoreMissingEmbeddedNotice>
       </properties>
     </project>
   </supplement>
@@ -522,8 +522,8 @@
       <groupId>com.amazonaws</groupId>
       <artifactId>jmespath-java</artifactId>
       <properties>
-        <license.ignoreMissingEmbeddedLicense>1.12.402</license.ignoreMissingEmbeddedLicense>
-        <license.ignoreMissingEmbeddedNotice>1.12.402</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreMissingEmbeddedLicense>1.12.637</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.12.637</license.ignoreMissingEmbeddedNotice>
       </properties>
     </project>
   </supplement>
@@ -533,8 +533,8 @@
       <groupId>com.amazonaws</groupId>
       <artifactId>aws-java-sdk-s3</artifactId>
       <properties>
-        <license.ignoreMissingEmbeddedLicense>1.12.402</license.ignoreMissingEmbeddedLicense>
-        <license.ignoreMissingEmbeddedNotice>1.12.402</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreMissingEmbeddedLicense>1.12.637</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.12.637</license.ignoreMissingEmbeddedNotice>
       </properties>
     </project>
   </supplement>
@@ -544,8 +544,8 @@
       <groupId>com.amazonaws</groupId>
       <artifactId>aws-java-sdk-kms</artifactId>
       <properties>
-        <license.ignoreMissingEmbeddedLicense>1.12.402</license.ignoreMissingEmbeddedLicense>
-        <license.ignoreMissingEmbeddedNotice>1.12.402</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreMissingEmbeddedLicense>1.12.637</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.12.637</license.ignoreMissingEmbeddedNotice>
       </properties>
     </project>
   </supplement>
@@ -555,8 +555,8 @@
       <groupId>com.amazonaws</groupId>
       <artifactId>aws-java-sdk-dynamodb</artifactId>
       <properties>
-        <license.ignoreMissingEmbeddedLicense>1.12.402</license.ignoreMissingEmbeddedLicense>
-        <license.ignoreMissingEmbeddedNotice>1.12.402</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreMissingEmbeddedLicense>1.12.637</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.12.637</license.ignoreMissingEmbeddedNotice>
       </properties>
     </project>
   </supplement>
@@ -609,10 +609,9 @@
       <groupId>org.reactivestreams</groupId>
       <artifactId>reactive-streams</artifactId>
       <properties>
-        <license.ignoreMissingEmbeddedLicense>1.0.3,1.0.4</license.ignoreMissingEmbeddedLicense>
-        <license.ignoreMissingEmbeddedNotice>1.0.3,1.0.4</license.ignoreMissingEmbeddedNotice>
-        <license.ignoreLicenseOverride>1.0.3,1.0.4</license.ignoreLicenseOverride>
-        <license.ignoreNoticeOverride>1.0.3</license.ignoreNoticeOverride>
+        <license.ignoreMissingEmbeddedLicense>1.0.4</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.0.4</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.0.4</license.ignoreLicenseOverride>
       </properties>
     </project>
   </supplement>
@@ -1131,6 +1130,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>
@@ -1163,6 +1187,19 @@
       <groupId>io.opencensus</groupId>
       <artifactId>opencensus-contrib-grpc-metrics</artifactId>
       <properties>
+        <license.ignoreMissingEmbeddedLicense>0.31.1</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.31.1</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.31.1</license.ignoreLicenseOverride>
+      </properties>
+    </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>
@@ -1293,6 +1330,226 @@
     </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.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-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>
+
+  <!-- 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.cloud:google-cloud-core is non-fixed ALv2 with no NOTICE file -->
   <supplement>
     <project>
@@ -1384,6 +1641,20 @@
   <supplement>
     <project>
       <groupId>io.grpc</groupId>
+      <artifactId>grpc-census</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.56.1</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.56.1</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.56.1</license.ignoreLicenseOverride>
+        <license.ignoreNoticeOverride>1.56.1</license.ignoreNoticeOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
       <artifactId>grpc-context</artifactId>
       <properties>
         <license.ignoreMissingEmbeddedLicense>1.56.1</license.ignoreMissingEmbeddedLicense>
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_qos-ch_slf4j_v_1.7.25_slf4j-api_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_qos-ch_slf4j_v_1.7.36_slf4j-api_LICENSE.txt
similarity index 95%
rename from asterixdb/src/main/licenses/content/raw.githubusercontent.com_qos-ch_slf4j_v_1.7.25_slf4j-api_LICENSE.txt
rename to asterixdb/src/main/licenses/content/raw.githubusercontent.com_qos-ch_slf4j_v_1.7.36_slf4j-api_LICENSE.txt
index 5a11c0c..f687729 100644
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_qos-ch_slf4j_v_1.7.25_slf4j-api_LICENSE.txt
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_qos-ch_slf4j_v_1.7.36_slf4j-api_LICENSE.txt
@@ -1,4 +1,4 @@
-Copyright (c) 2004-2007 QOS.ch
+Copyright (c) 2004-2022 QOS.ch Sarl (Switzerland)
 All rights reserved.
 
 Permission is hereby granted, free  of charge, to any person obtaining
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/asterixdb/src/main/licenses/content/spdx.org_licenses_MIT-0.txt b/asterixdb/src/main/licenses/content/spdx.org_licenses_MIT-0.txt
new file mode 100644
index 0000000..a4e9dc9
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/spdx.org_licenses_MIT-0.txt
@@ -0,0 +1,16 @@
+MIT No Attribution
+
+Copyright <YEAR> <COPYRIGHT HOLDER>
+
+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.
+
+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/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 85910aa..a44e1be 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,6 +18,7 @@
  */
 package org.apache.hyracks.algebricks.compiler.api;
 
+import java.util.EnumSet;
 import java.util.List;
 
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -47,6 +48,7 @@
 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.JobFlag;
 import org.apache.hyracks.api.job.JobSpecification;
 
 public class HeuristicCompilerFactoryBuilder extends AbstractCompilerFactoryBuilder {
@@ -172,6 +174,17 @@
         }
 
         @Override
+        public JobSpecification createJob(Object appContext, IJobletEventListenerFactory jobEventListenerFactory,
+                EnumSet<JobFlag> runtimeFlags) throws AlgebricksException {
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Starting Job Generation.\n");
+            PlanCompiler pc = factory.createPlanCompiler(oc, appContext, writerFactory);
+            if (runtimeFlags.contains(JobFlag.PROFILE_RUNTIME)) {
+                pc.enableLog2PhysMapping();
+            }
+            return pc.compilePlan(plan, jobEventListenerFactory);
+        }
+
+        @Override
         public boolean skipJobCapacityAssignment() {
             return oc.skipJobCapacityAssignment();
         }
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompiler.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompiler.java
index 0f5798e..7de0adb 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompiler.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompiler.java
@@ -18,8 +18,11 @@
  */
 package org.apache.hyracks.algebricks.compiler.api;
 
+import java.util.EnumSet;
+
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.job.IJobletEventListenerFactory;
+import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.api.job.JobSpecification;
 
 public interface ICompiler {
@@ -28,5 +31,8 @@
     public JobSpecification createJob(Object appContext, IJobletEventListenerFactory jobEventListenerFactory)
             throws AlgebricksException;
 
+    JobSpecification createJob(Object appContext, IJobletEventListenerFactory jobletEventListenerFactory,
+            EnumSet<JobFlag> runtimeFlags) throws AlgebricksException;
+
     boolean skipJobCapacityAssignment();
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/pom.xml b/hyracks-fullstack/algebricks/algebricks-core/pom.xml
index d1294e2..a67dd71 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-core/pom.xml
@@ -111,5 +111,9 @@
       <artifactId>junit</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
index 4466408..9c41cb9 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
@@ -46,6 +46,7 @@
     SELECT,
     SINK,
     SPLIT,
+    SWITCH,
     SUBPLAN,
     TOKENIZE,
     UNIONALL,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
index 4a900af..c052d58 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -68,6 +68,7 @@
     SORT_MERGE_EXCHANGE,
     SPATIAL_JOIN,
     SPLIT,
+    SWITCH,
     STABLE_SORT,
     STATS,
     STREAM_LIMIT,
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 cecbaf7..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
@@ -130,7 +130,7 @@
 
         @Override
         public String toString() {
-            return "NULL";
+            return "null";
         }
 
         @Override
@@ -167,7 +167,7 @@
 
         @Override
         public String toString() {
-            return "MISSING";
+            return "missing";
         }
 
         @Override
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/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/SwitchOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SwitchOperator.java
new file mode 100644
index 0000000..4b1f260
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SwitchOperator.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * 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 from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.operators.logical;
+
+import java.util.Map;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+
+/**
+ * Switch Operator receives an expression and an output mapping. We evaluate the expression during runtime and look up
+ * the result in the output mapping. Based on this, we propagate each tuple to the corresponding output branch(es).
+ */
+public class SwitchOperator extends AbstractReplicateOperator {
+
+    // Expression containing the index of the relevant field
+    private final Mutable<ILogicalExpression> branchingExpression;
+
+    // The supplied mapping from field values to arrays of output branch numbers
+    private final Map<Integer, int[]> outputMapping;
+
+    public SwitchOperator(int outputArity, Mutable<ILogicalExpression> branchingExpression,
+            Map<Integer, int[]> outputMapping) {
+        super(outputArity);
+        this.branchingExpression = branchingExpression;
+        this.outputMapping = outputMapping;
+    }
+
+    @Override
+    public LogicalOperatorTag getOperatorTag() {
+        return LogicalOperatorTag.SWITCH;
+    }
+
+    @Override
+    public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+        return visitor.visitSwitchOperator(this, arg);
+    }
+
+    public Mutable<ILogicalExpression> getBranchingExpression() {
+        return branchingExpression;
+    }
+
+    public Map<Integer, int[]> getOutputMapping() {
+        return outputMapping;
+    }
+
+    @Override
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+        return visitor.transform(branchingExpression);
+    }
+}
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/logical/visitors/CardinalityInferenceVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
index 6ed90a5..14221d6 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
@@ -58,6 +58,7 @@
 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.SwitchOperator;
 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;
@@ -196,6 +197,11 @@
     }
 
     @Override
+    public Long visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+        return op.getInputs().get(0).getValue().accept(this, arg);
+    }
+
+    @Override
     public Long visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
         return op.getInputs().get(0).getValue().accept(this, arg);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
index f3717c8..5937d9d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
@@ -74,6 +74,7 @@
 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.SwitchOperator;
 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;
@@ -427,6 +428,12 @@
     }
 
     @Override
+    public Void visitSwitchOperator(SwitchOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        propagateFDsAndEquivClasses(op, ctx);
+        return null;
+    }
+
+    @Override
     public Void visitMaterializeOperator(MaterializeOperator op, IOptimizationContext ctx) throws AlgebricksException {
         propagateFDsAndEquivClasses(op, ctx);
         return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index 9e2e87c..cfae695 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -27,6 +27,7 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -66,6 +67,7 @@
 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.SwitchOperator;
 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;
@@ -310,6 +312,12 @@
     }
 
     @Override
+    public Boolean visitSwitchOperator(SwitchOperator op, ILogicalOperator arg) throws AlgebricksException {
+        // TODO (GLENN): Implement this logic
+        throw new NotImplementedException();
+    }
+
+    @Override
     public Boolean visitMaterializeOperator(MaterializeOperator op, ILogicalOperator arg) throws AlgebricksException {
         AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
         if (aop.getOperatorTag() != LogicalOperatorTag.MATERIALIZE) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
index 8ca2b83..e4d6586 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
@@ -65,6 +65,7 @@
 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.SwitchOperator;
 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;
@@ -205,6 +206,12 @@
     }
 
     @Override
+    public Void visitSwitchOperator(SwitchOperator op, ILogicalOperator arg) throws AlgebricksException {
+        mapVariablesStandard(op, arg);
+        return null;
+    }
+
+    @Override
     public Void visitMaterializeOperator(MaterializeOperator op, ILogicalOperator arg) throws AlgebricksException {
         mapVariablesStandard(op, arg);
         return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
index e242531..08acf13 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
@@ -28,6 +28,7 @@
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -64,6 +65,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 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.SwitchOperator;
 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;
@@ -447,6 +449,12 @@
     }
 
     @Override
+    public ILogicalOperator visitSwitchOperator(SwitchOperator op, ILogicalOperator arg) throws AlgebricksException {
+        // TODO (GLENN): Implement this logic
+        throw new NotImplementedException();
+    }
+
+    @Override
     public ILogicalOperator visitMaterializeOperator(MaterializeOperator op, ILogicalOperator arg)
             throws AlgebricksException {
         MaterializeOperator opCopy = new MaterializeOperator();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
index 5a566ee..3a88d2c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
@@ -57,6 +57,7 @@
 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.SwitchOperator;
 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;
@@ -194,6 +195,11 @@
     }
 
     @Override
+    public Void visitSwitchOperator(SwitchOperator op, IOptimizationContext arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
     public Void visitMaterializeOperator(MaterializeOperator op, IOptimizationContext arg) throws AlgebricksException {
         // TODO Auto-generated method stub
         return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
index c2ee661..8ef0b5b 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
@@ -20,6 +20,7 @@
 package org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
@@ -63,6 +64,7 @@
 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.SwitchOperator;
 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;
@@ -176,7 +178,13 @@
 
     @Override
     public ILogicalOperator visitSplitOperator(SplitOperator op, Void arg) throws AlgebricksException {
-        return new SplitOperator(op.getOutputArity(), op.getBranchingExpression());
+        return new SplitOperator(op.getOutputArity(), deepCopyExpressionRef(op.getBranchingExpression()));
+    }
+
+    @Override
+    public ILogicalOperator visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+        return new SwitchOperator(op.getOutputArity(), deepCopyExpressionRef(op.getBranchingExpression()),
+                new HashMap<>(op.getOutputMapping()));
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
index eb90288..8a3a885 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
@@ -58,6 +58,7 @@
 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.SwitchOperator;
 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;
@@ -186,6 +187,11 @@
     }
 
     @Override
+    public Void visitSwitchOperator(SwitchOperator op, IOptimizationContext arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
     public Void visitMaterializeOperator(MaterializeOperator op, IOptimizationContext ctx) throws AlgebricksException {
         return null;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
index 5d9d7895..ff50994 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
@@ -62,6 +62,7 @@
 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.SwitchOperator;
 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;
@@ -271,6 +272,11 @@
     }
 
     @Override
+    public Void visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
     public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
         return null;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
index 44bb7e2..e7d6a92 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
@@ -60,6 +60,7 @@
 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.SwitchOperator;
 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;
@@ -290,6 +291,12 @@
     }
 
     @Override
+    public Void visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
     public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
         standardLayout(op);
         return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
index 439e493..cf8196c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -23,6 +23,7 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -63,6 +64,7 @@
 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.SwitchOperator;
 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;
@@ -410,6 +412,13 @@
     }
 
     @Override
+    public Void visitSwitchOperator(SwitchOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+            throws AlgebricksException {
+        // TODO (GLENN): Implement this logic
+        throw new NotImplementedException();
+    }
+
+    @Override
     public Void visitMaterializeOperator(MaterializeOperator op, Pair<LogicalVariable, LogicalVariable> pair)
             throws AlgebricksException {
         // does not produce/use any variables
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index 174b184..4c994b5 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -25,6 +25,7 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -64,6 +65,7 @@
 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.SwitchOperator;
 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;
@@ -473,6 +475,12 @@
     }
 
     @Override
+    public Void visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+        // TODO (GLENN): Implement this logic
+        throw new NotImplementedException();
+    }
+
+    @Override
     public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
         return null;
     }
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/SwitchPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SwitchPOperator.java
new file mode 100644
index 0000000..4b58788
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SwitchPOperator.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.algebricks.core.algebra.operators.physical;
+
+import java.util.Map;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SwitchOperator;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.operators.std.SwitchOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+
+public class SwitchPOperator extends AbstractReplicatePOperator {
+
+    @Override
+    public PhysicalOperatorTag getOperatorTag() {
+        return PhysicalOperatorTag.SWITCH;
+    }
+
+    @Override
+    public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+            IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+            throws AlgebricksException {
+        SwitchOperator sop = (SwitchOperator) op;
+        int outputArity = sop.getOutputArity();
+        Map<Integer, int[]> outputMapping = sop.getOutputMapping();
+
+        IOperatorDescriptorRegistry spec = builder.getJobSpec();
+        RecordDescriptor recDescriptor =
+                JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
+
+        IExpressionRuntimeProvider expressionRuntimeProvider = context.getExpressionRuntimeProvider();
+        IScalarEvaluatorFactory branchingExprEvalFactory = expressionRuntimeProvider.createEvaluatorFactory(
+                sop.getBranchingExpression().getValue(), context.getTypeEnvironment(op), inputSchemas, context);
+
+        IBinaryIntegerInspectorFactory intInspectorFactory = context.getBinaryIntegerInspectorFactory();
+
+        SwitchOperatorDescriptor sopDesc = new SwitchOperatorDescriptor(spec, recDescriptor, outputArity,
+                branchingExprEvalFactory, intInspectorFactory, outputMapping);
+        sopDesc.setSourceLocation(sop.getSourceLocation());
+
+        contributeOpDesc(builder, sop, sopDesc);
+        ILogicalOperator src = op.getInputs().get(0).getValue();
+        builder.contributeGraphEdge(src, 0, op, 0);
+    }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/IPlanPrettyPrinter.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/IPlanPrettyPrinter.java
index 66d48d3..9f138b5 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,11 +18,15 @@
  */
 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;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 /**
  * Note: Some implementations may be stateful and not thread-safe.
  */
@@ -43,6 +47,14 @@
     /** Prints the whole logical plan. */
     IPlanPrettyPrinter printPlan(ILogicalPlan plan, boolean printOptimizerEstimates) throws AlgebricksException;
 
+    /** Prints the logical plan, annotated with physical operator and connector ids */
+    IPlanPrettyPrinter printPlan(ILogicalPlan plan, Map<Object, String> log2phys, boolean printOptimizerEstimates)
+            throws AlgebricksException;
+
+    /** Prints the logical plan, annotated with physical operator and connector ids, and profiling info*/
+    IPlanPrettyPrinter printPlan(ILogicalPlan plan, Map<Object, String> log2phys, boolean printOptimizerEstimates,
+            ObjectNode profile) 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 a60c969..91217f1 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,9 +20,11 @@
 
 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;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -65,6 +67,7 @@
 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.SwitchOperator;
 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;
@@ -73,6 +76,8 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class LogicalOperatorPrettyPrintVisitor extends AbstractLogicalOperatorPrettyPrintVisitor<Integer>
         implements IPlanPrettyPrinter {
 
@@ -97,6 +102,22 @@
     }
 
     @Override
+    public final IPlanPrettyPrinter printPlan(ILogicalPlan plan, Map<Object, String> log2phys,
+            boolean printOptimizerEstimates) throws AlgebricksException {
+        //TODO(ian): would be nice if the text plan returned real operator ids too
+        printPlanImpl(plan, 0, printOptimizerEstimates);
+        return this;
+    }
+
+    @Override
+    public IPlanPrettyPrinter printPlan(ILogicalPlan plan, Map<Object, String> log2phys,
+            boolean printOptimizerEstimates, ObjectNode profile) throws AlgebricksException {
+        //TODO(ian): add times
+        printPlanImpl(plan, 0, printOptimizerEstimates);
+        return this;
+    }
+
+    @Override
     public final IPlanPrettyPrinter printOperator(AbstractLogicalOperator op, boolean printInputs,
             boolean printOptimizerEstimates) throws AlgebricksException {
         printOperatorImpl(op, 0, printInputs, printOptimizerEstimates);
@@ -453,6 +474,12 @@
     }
 
     @Override
+    public Void visitSwitchOperator(SwitchOperator op, Integer indent) throws AlgebricksException {
+        // TODO (GLENN): Implement this logic
+        throw new NotImplementedException();
+    }
+
+    @Override
     public Void visitMaterializeOperator(MaterializeOperator op, Integer indent) throws AlgebricksException {
         addIndent(indent).append("materialize");
         return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
index 464d15e..5c532d3 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,14 +19,17 @@
 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;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -72,6 +75,7 @@
 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.SwitchOperator;
 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;
@@ -79,12 +83,15 @@
 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.api.dataflow.OperatorDescriptorId;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 
 import com.fasterxml.jackson.core.JsonFactory;
 import com.fasterxml.jackson.core.JsonGenerator;
 import com.fasterxml.jackson.core.util.DefaultIndenter;
 import com.fasterxml.jackson.core.util.DefaultPrettyPrinter;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class LogicalOperatorPrettyPrintVisitorJson extends AbstractLogicalOperatorPrettyPrintVisitor<Void>
         implements IPlanPrettyPrinter {
@@ -100,6 +107,8 @@
     private static final String MISSING_VALUE_FIELD = "missing-value";
     private static final String OPTIMIZER_ESTIMATES = "optimizer-estimates";
     private final Map<AbstractLogicalOperator, String> operatorIdentity = new HashMap<>();
+    private Map<Object, String> log2odid = Collections.emptyMap();
+    private Map<String, OperatorProfile> profile = Collections.emptyMap();
     private final IdCounter idCounter = new IdCounter();
     private final JsonGenerator jsonGenerator;
 
@@ -147,6 +156,124 @@
         }
     }
 
+    private class ExtendedActivityId {
+        private final OperatorDescriptorId odId;
+        private final int id;
+        private final int microId;
+        private final int subPipe;
+        private final int subId;
+
+        ExtendedActivityId(String str) {
+            if (str.startsWith("ANID:")) {
+                str = str.substring(5);
+                int idIdx = str.lastIndexOf(':');
+                this.odId = OperatorDescriptorId.parse(str.substring(0, idIdx));
+                String[] parts = str.substring(idIdx + 1).split("\\.");
+                this.id = Integer.parseInt(parts[0]);
+                if (parts.length >= 2) {
+                    this.microId = Integer.parseInt(parts[1]);
+                } else {
+                    this.microId = -1;
+                }
+                if (parts.length >= 4) {
+                    this.subPipe = Integer.parseInt(parts[2]);
+                    this.subId = Integer.parseInt(parts[3]);
+                } else {
+                    this.subPipe = -1;
+                    this.subId = -1;
+                }
+            } else {
+                throw new IllegalArgumentException("Unable to parse: " + str);
+            }
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(values());
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            return (o instanceof ExtendedActivityId) && Objects.equals(((ExtendedActivityId) o).values(), values());
+        }
+
+        private List<?> values() {
+            return List.of(odId, id, microId, subPipe, subId);
+        }
+
+        @Override
+        public String toString() {
+            return "ANID:" + odId + ":" + getLocalId();
+        }
+
+        private void catenateId(StringBuilder sb, int i) {
+            if (sb.length() == 0) {
+                sb.append(i);
+                return;
+            }
+            sb.append(".");
+            sb.append(i);
+        }
+
+        public String getLocalId() {
+            StringBuilder sb = new StringBuilder();
+            catenateId(sb, odId.getId());
+            if (microId > 0) {
+                catenateId(sb, microId);
+            }
+            if (subId > 0) {
+                catenateId(sb, subPipe);
+                catenateId(sb, subId);
+            }
+            return sb.toString();
+        }
+    }
+
+    private class OperatorProfile {
+        Map<String, Pair<Double, Double>> activities;
+
+        OperatorProfile() {
+            activities = new HashMap<>();
+        }
+
+        void updateOperator(String extendedOpId, double time) {
+            Pair<Double, Double> times = activities.computeIfAbsent(extendedOpId, i -> new Pair(time, time));
+            if (times.getFirst() > time) {
+                times.setFirst(time);
+            }
+            if (times.getSecond() < time) {
+                times.setSecond(time);
+            }
+        }
+    }
+
+    private ExtendedActivityId acIdFromName(String name) {
+        String[] parts = name.split(" - ");
+        return new ExtendedActivityId(parts[0]);
+    }
+
+    Map<String, OperatorProfile> processProfile(ObjectNode profile) {
+        Map<String, OperatorProfile> profiledOps = new HashMap<>();
+        for (JsonNode joblet : profile.get("joblets")) {
+            for (JsonNode task : joblet.get("tasks")) {
+                for (JsonNode counters : task.get("counters")) {
+                    OperatorProfile info = profiledOps.computeIfAbsent(counters.get("runtime-id").asText(),
+                            i -> new OperatorProfile());
+                    info.updateOperator(acIdFromName(counters.get("name").asText()).getLocalId(),
+                            counters.get("run-time").asDouble());
+                }
+                for (JsonNode partition : task.get("partition-send-profile")) {
+                    String id = partition.get("partition-id").get("connector-id").asText();
+                    OperatorProfile info = profiledOps.computeIfAbsent(id, i -> new OperatorProfile());
+                    //CDIDs are unique
+                    info.updateOperator("0",
+                            partition.get("close-time").asDouble() - partition.get("open-time").asDouble());
+                }
+            }
+        }
+        return profiledOps;
+    }
+
     @Override
     public final IPlanPrettyPrinter reset() throws AlgebricksException {
         flushContentToWriter();
@@ -164,7 +291,25 @@
     }
 
     @Override
+    public final IPlanPrettyPrinter printPlan(ILogicalPlan plan, Map<Object, String> log2phys,
+            boolean printOptimizerEstimates) throws AlgebricksException {
+        this.log2odid = log2phys;
+        printPlanImpl(plan, printOptimizerEstimates);
+        flushContentToWriter();
+        return this;
+    }
 
+    @Override
+    public IPlanPrettyPrinter printPlan(ILogicalPlan plan, Map<Object, String> log2phys,
+            boolean printOptimizerEstimates, ObjectNode profile) throws AlgebricksException {
+        this.log2odid = log2phys;
+        this.profile = processProfile(profile);
+        printPlanImpl(plan, printOptimizerEstimates);
+        flushContentToWriter();
+        return this;
+    }
+
+    @Override
     public final IPlanPrettyPrinter printOperator(AbstractLogicalOperator op, boolean printInputs,
             boolean printOptimizerEstimates) throws AlgebricksException {
         printOperatorImpl(op, printInputs, printOptimizerEstimates);
@@ -195,6 +340,28 @@
             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);
+                OperatorProfile info = profile.get(od);
+                if (info != null) {
+                    if (info.activities.size() == 1) {
+                        Pair<Double, Double> minMax = info.activities.values().iterator().next();
+                        jsonGenerator.writeNumberField("min-time", minMax.first);
+                        jsonGenerator.writeNumberField("max-time", minMax.second);
+                    } else {
+                        jsonGenerator.writeObjectFieldStart("times");
+                        for (Map.Entry<String, Pair<Double, Double>> ac : info.activities.entrySet()) {
+                            jsonGenerator.writeObjectFieldStart(ac.getKey());
+                            jsonGenerator.writeNumberField("min-time", ac.getValue().first);
+                            jsonGenerator.writeNumberField("max-time", ac.getValue().second);
+                            jsonGenerator.writeEndObject();
+                        }
+                        jsonGenerator.writeEndObject();
+                    }
+
+                }
+            }
             IPhysicalOperator pOp = op.getPhysicalOperator();
             if (pOp != null) {
                 jsonGenerator.writeStringField("physical-operator", pOp.toString(false));
@@ -631,6 +798,12 @@
     }
 
     @Override
+    public Void visitSwitchOperator(SwitchOperator op, Void indent) throws AlgebricksException {
+        // TODO (GLENN): Implement this logic
+        throw new NotImplementedException();
+    }
+
+    @Override
     public Void visitMaterializeOperator(MaterializeOperator op, Void indent) throws AlgebricksException {
         try {
             jsonGenerator.writeStringField(OPERATOR_FIELD, "materialize");
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
index d521831..4eb6494 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
@@ -48,6 +48,7 @@
 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.SwitchOperator;
 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;
@@ -88,6 +89,8 @@
 
     public R visitSplitOperator(SplitOperator op, T arg) throws AlgebricksException;
 
+    public R visitSwitchOperator(SwitchOperator op, T arg) throws AlgebricksException;
+
     public R visitMaterializeOperator(MaterializeOperator op, T arg) throws AlgebricksException;
 
     public R visitScriptOperator(ScriptOperator op, T arg) throws AlgebricksException;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/LogicalExpressionReferenceTransformVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/LogicalExpressionReferenceTransformVisitor.java
index 9350f95..f1613a5 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/LogicalExpressionReferenceTransformVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/LogicalExpressionReferenceTransformVisitor.java
@@ -50,6 +50,7 @@
 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.SwitchOperator;
 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;
@@ -166,6 +167,12 @@
     }
 
     @Override
+    public Boolean visitSwitchOperator(SwitchOperator op, ILogicalExpressionReferenceTransform arg)
+            throws AlgebricksException {
+        return visitOperator(op, arg);
+    }
+
+    @Override
     public Boolean visitMaterializeOperator(MaterializeOperator op, ILogicalExpressionReferenceTransform arg)
             throws AlgebricksException {
         return visitOperator(op, arg);
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 e53322c..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
@@ -44,4 +44,5 @@
     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..ca26515 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,11 +21,13 @@
 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;
 
 import org.apache.commons.lang3.ArrayUtils;
+import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksCountPartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -37,6 +39,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
@@ -71,6 +74,8 @@
 
     private int aodCounter = 0;
 
+    private boolean genLog2PhysMap = false;
+
     public JobBuilder(JobSpecification jobSpec, AlgebricksAbsolutePartitionConstraint clusterLocations) {
         this.jobSpec = jobSpec;
         this.clusterLocations = clusterLocations;
@@ -93,6 +98,10 @@
                 new String[] { clusterLocations.getLocations()[Math.abs(jobSpec.hashCode() % nPartitions)] });
     }
 
+    public void enableLog2PhysMapping() {
+        this.genLog2PhysMap = true;
+    }
+
     @Override
     public void contributeMicroOperator(ILogicalOperator op, IPushRuntimeFactory runtime, RecordDescriptor recDesc) {
         contributeMicroOperator(op, runtime, recDesc, null);
@@ -147,6 +156,43 @@
         hyracksOps.put(op, opDesc);
     }
 
+    private String getExtendedOdidForMetaOp(ILogicalOperator op, int k) {
+        String base = metaAsterixOps.get(k).getOperatorId().toString();
+        Pair<IPushRuntimeFactory, RecordDescriptor> fact = microOps.get(op);
+        List<Pair<IPushRuntimeFactory, RecordDescriptor>> metaOpPipeline = metaAsterixOpSkeletons.get(k);
+        int pos = metaOpPipeline.indexOf(fact);
+        return base + "." + pos;
+    }
+
+    private void getExtendedOdidForSubplanOp(SubplanOperator op, Map<ILogicalOperator, String> log2phys) {
+        String baseId = getExtendedOdidForMetaOp(op, algebraicOpBelongingToMetaAsterixOp.get(op));
+        op.getNestedPlans().forEach(plan -> plan.getRoots()
+                .forEach(root -> getExtendedOdidForOperator(baseId, root.getValue(), log2phys, 0)));
+    }
+
+    private int getExtendedOdidForOperator(String baseId, ILogicalOperator op, Map<ILogicalOperator, String> log2phys,
+            int input) {
+        List<Mutable<ILogicalOperator>> inputs = op.getInputs();
+        List<Integer> paths = new ArrayList<>(inputs.size());
+        for (int i = 0; i < inputs.size(); i++) {
+            ILogicalOperator nextOp = inputs.get(i).getValue();
+            paths.add(i, getExtendedOdidForOperator(baseId, nextOp, log2phys, i + input));
+        }
+        int lPath = paths.size() > 0 ? Collections.max(paths) : 0;
+        log2phys.put(op, baseId + "." + input + "." + lPath);
+        return lPath + 1;
+    }
+
+    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, getExtendedOdidForMetaOp(k, v)));
+        microOps.keySet().stream().filter(op -> op instanceof SubplanOperator)
+                .forEach(op -> getExtendedOdidForSubplanOp((SubplanOperator) op, mergedOperatorMap));
+        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 +220,11 @@
             jobSpec.addRoot(opDesc);
         }
         setAllPartitionConstraints(tgtConstraints);
+        if (genLog2PhysMap) {
+            jobSpec.setLogical2PhysicalMap(getLogical2PhysicalMap());
+        } else {
+            jobSpec.setLogical2PhysicalMap(Collections.emptyMap());
+        }
     }
 
     public List<IOperatorDescriptor> getGeneratedMetaOps() {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/PlanCompiler.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/PlanCompiler.java
index 2ef5c6c..f8bdfa5 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/PlanCompiler.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/PlanCompiler.java
@@ -39,6 +39,8 @@
     private JobGenContext context;
     private Map<Mutable<ILogicalOperator>, List<Mutable<ILogicalOperator>>> operatorVisitedToParents = new HashMap<>();
 
+    boolean genLog2PhysMap = false;
+
     public PlanCompiler(JobGenContext context) {
         this.context = context;
     }
@@ -47,6 +49,10 @@
         return context;
     }
 
+    public void enableLog2PhysMapping() {
+        this.genLog2PhysMap = true;
+    }
+
     public JobSpecification compilePlan(ILogicalPlan plan, IJobletEventListenerFactory jobEventListenerFactory)
             throws AlgebricksException {
         return compilePlanImpl(plan, false, null, jobEventListenerFactory);
@@ -66,6 +72,9 @@
         }
         List<ILogicalOperator> rootOps = new ArrayList<>();
         JobBuilder builder = new JobBuilder(spec, context.getClusterLocations());
+        if (genLog2PhysMap) {
+            builder.enableLog2PhysMapping();
+        }
         for (Mutable<ILogicalOperator> opRef : plan.getRoots()) {
             compileOpRef(opRef, spec, builder, outerPlanSchema);
             rootOps.add(opRef.getValue());
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 fb77ba0..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,7 @@
     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";
@@ -281,6 +282,14 @@
         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);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
index ee7f7aa..75da5ff 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
@@ -26,6 +26,7 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -63,6 +64,7 @@
 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.SwitchOperator;
 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;
@@ -484,6 +486,12 @@
     }
 
     @Override
+    public String visitSwitchOperator(SwitchOperator op, Boolean showDetails) throws AlgebricksException {
+        // TODO (GLENN): Implement this logic
+        throw new NotImplementedException();
+    }
+
+    @Override
     public String visitMaterializeOperator(MaterializeOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("materialize");
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateLeftOuterJoinSelectsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateLeftOuterJoinSelectsRule.java
new file mode 100644
index 0000000..3114524
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateLeftOuterJoinSelectsRule.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+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.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.expressions.IAlgebricksConstantValue;
+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.functions.IFunctionInfo;
+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.SelectOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * In case {@link LeftOuterJoinOperator} gets transformed into {@link LeftOuterUnnestMapOperator}, the plan could
+ * end up with multiple SELECT operators (broken up by {@link BreakSelectIntoConjunctsRule}).
+ * This rule consolidate those SELECT operators back again.
+ * <p>
+ * Example:
+ * select (and([JOIN_CONDITION], eq($$o.getField("x"), 1)))
+ * -- unnest $$o <- dataset(...)
+ * <p>
+ * After {@link BreakSelectIntoConjunctsRule}
+ * select ([JOIN_CONDITION])
+ * -- select (eq($$o.getField("x"), 1))
+ * -- -- unnest $$o <- dataset(...)
+ * <p>
+ * Before accessMethod rewrite:
+ * left outer join ([JOIN_CONDITION])
+ * -- data-scan []<-[$$56, $$c] <- ...
+ * -- select (eq($$o.getField("x"), 1))
+ * -- -- data-scan []<-[$$57, $$o] <- ...
+ * <p>
+ * After accessMethod rewrite:
+ * select ([JOIN_CONDITION]) retain-untrue (... <- missing)
+ * -- select (eq($$o.getField("x"), 1))
+ * -- -- left-outer-unnest-map ...
+ * -- -- -- ...
+ * <p>
+ * After this rule:
+ * select (and([JOIN_CONDITION], eq($$o.getField("x"), 1))) retain-untrue (... <- missing)
+ * -- left-outer-unnest-map ...
+ */
+public class ConsolidateLeftOuterJoinSelectsRule implements IAlgebraicRewriteRule {
+    private final List<Mutable<ILogicalExpression>> conditions = new ArrayList<>();
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        ILogicalOperator op = opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+            return false;
+        }
+
+        conditions.clear();
+        IAlgebricksConstantValue missingValue = null;
+        LogicalVariable missingVariable = null;
+        ILogicalOperator nextOp = op;
+        do {
+            SelectOperator selectOp = (SelectOperator) nextOp;
+            if (missingValue == null) {
+                // Capture the SELECT that contains the retain missing value and variable placeholder
+                missingValue = selectOp.getRetainMissingAsValue();
+                missingVariable = selectOp.getMissingPlaceholderVariable();
+            }
+            conditions.add(new MutableObject<>(selectOp.getCondition().getValue()));
+            nextOp = nextOp.getInputs().get(0).getValue();
+        } while (nextOp.getOperatorTag() == LogicalOperatorTag.SELECT);
+
+        if (conditions.size() < 2 || missingValue == null) {
+            return false;
+        }
+
+        SelectOperator newSelect = new SelectOperator(createAndCondition(context), missingValue, missingVariable);
+        newSelect.getInputs().add(new MutableObject<>(nextOp));
+        opRef.setValue(newSelect);
+        context.computeAndSetTypeEnvironmentForOperator(newSelect);
+        return true;
+    }
+
+    private Mutable<ILogicalExpression> createAndCondition(IOptimizationContext context) {
+        IFunctionInfo fInfo = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND);
+        return new MutableObject<>(new ScalarFunctionCallExpression(fInfo, new ArrayList<>(conditions)));
+    }
+}
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/PushMapOperatorDownThroughProductRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushMapOperatorDownThroughProductRule.java
index 2a009f0..5d466d5 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushMapOperatorDownThroughProductRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushMapOperatorDownThroughProductRule.java
@@ -75,6 +75,8 @@
             op2Ref.setValue(b0);
             b0Ref.setValue(op1);
             opRef.setValue(op2);
+            context.computeAndSetTypeEnvironmentForOperator(op1);
+            context.computeAndSetTypeEnvironmentForOperator(op2);
             return true;
         } else {
             Mutable<ILogicalOperator> b1Ref = op2.getInputs().get(1);
@@ -86,6 +88,8 @@
                 op2Ref.setValue(b1);
                 b1Ref.setValue(op1);
                 opRef.setValue(op2);
+                context.computeAndSetTypeEnvironmentForOperator(op1);
+                context.computeAndSetTypeEnvironmentForOperator(op2);
                 return true;
             } else {
                 return false;
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
index a6fe495..a1aa01a 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
@@ -73,6 +73,7 @@
 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.SwitchOperator;
 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;
@@ -112,6 +113,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.StreamSelectPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.StringStreamingScriptPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.SubplanPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.SwitchPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.TokenizePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.UnionAllPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.UnnestPOperator;
@@ -309,6 +311,11 @@
         }
 
         @Override
+        public IPhysicalOperator visitSwitchOperator(SwitchOperator op, Boolean topLevelOp) {
+            return new SwitchPOperator();
+        }
+
+        @Override
         public IPhysicalOperator visitScriptOperator(ScriptOperator op, Boolean topLevelOp) {
             return new StringStreamingScriptPOperator();
         }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetMemoryRequirementsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetMemoryRequirementsRule.java
index 3b0e9fb..b3748dd 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetMemoryRequirementsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetMemoryRequirementsRule.java
@@ -59,6 +59,7 @@
 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.SwitchOperator;
 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;
@@ -272,6 +273,11 @@
         }
 
         @Override
+        public Void visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+            return null;
+        }
+
+        @Override
         public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
             return null;
         }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
index 1388ccb..f4f9eda 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
@@ -56,6 +56,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 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.SwitchOperator;
 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;
@@ -186,6 +187,11 @@
     }
 
     @Override
+    public ILogicalOperator visitSwitchOperator(SwitchOperator op, Void arg) throws AlgebricksException {
+        return visit(op);
+    }
+
+    @Override
     public ILogicalOperator visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
         return visit(op);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
index 82b6f9c..b1bd46b 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/IPushRuntimeFactory.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public interface IPushRuntimeFactory extends Serializable {
+
     IPushRuntime[] createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException;
 
     default void setSourceLocation(SourceLocation sourceLoc) {
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/ProfiledPushRuntime.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/ProfiledPushRuntime.java
new file mode 100644
index 0000000..354f172
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/base/ProfiledPushRuntime.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.algebricks.runtime.base;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.dataflow.ITimedWriter;
+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.profiling.IOperatorStats;
+
+public class ProfiledPushRuntime extends ProfiledFrameWriter implements IPushRuntime {
+
+    private final IPushRuntime wrapped;
+    private final IOperatorStats stats;
+
+    private final boolean last;
+
+    private final Map<Integer, ITimedWriter> outputs;
+
+    public ProfiledPushRuntime(IPushRuntime push, IOperatorStats stats, boolean last) {
+        super(push);
+        outputs = new HashMap<>();
+        this.wrapped = push;
+        this.stats = stats;
+        this.last = last;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        super.close();
+        long ownTime = getTotalTime();
+        //for micro union all. accumulate the time of each input into the counter.
+        //then, on input 0, subtract the output from the accumulated time.
+        if (!last) {
+            stats.getTimeCounter().update(ownTime);
+            return;
+        }
+        ownTime += stats.getTimeCounter().get();
+        for (ITimedWriter w : outputs.values()) {
+            ownTime -= w.getTotalTime();
+        }
+        stats.getTimeCounter().set(ownTime);
+    }
+
+    public IOperatorStats getStats() {
+        return stats;
+    }
+
+    @Override
+    public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+        if (writer instanceof ITimedWriter) {
+            ITimedWriter wrapper = (ITimedWriter) writer;
+            wrapper.setUpstreamStats(stats);
+            outputs.put(index, wrapper);
+        }
+        wrapped.setOutputFrameWriter(index, writer, recordDesc);
+    }
+
+    @Override
+    public void setInputRecordDescriptor(int index, RecordDescriptor recordDescriptor) {
+        wrapped.setInputRecordDescriptor(index, recordDescriptor);
+    }
+
+    public static IPushRuntime time(IPushRuntime push, IOperatorStats stats, boolean last) throws HyracksDataException {
+        if (!(push instanceof ProfiledPushRuntime)) {
+            return new ProfiledPushRuntime(push, stats, last);
+        } else {
+            return push;
+        }
+    }
+
+    public static IPushRuntime time(IPushRuntime push, IOperatorStats stats) throws HyracksDataException {
+        return time(push, stats, true);
+    }
+}
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/meta/AlgebricksMetaOperatorDescriptor.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java
index b3fef7f..2a3fa7e 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java
@@ -19,26 +19,34 @@
 package org.apache.hyracks.algebricks.runtime.operators.meta;
 
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.ITimedWriter;
 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.NoOpOperatorStats;
+import org.apache.hyracks.api.job.profiling.OperatorStats;
 import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputIntrospectingOperatorNodePushable;
 import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
 
 import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class AlgebricksMetaOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
 
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 3L;
 
     // array of factories for building the local runtime pipeline
     private final AlgebricksPipeline pipeline;
@@ -85,6 +93,68 @@
         }
     }
 
+    private static String makeStatName(String base, String name, int pos, int input, int subPlan, int subPos) {
+        StringBuilder sb = new StringBuilder();
+        sb.append(base);
+        sb.append(".");
+        sb.append(pos);
+        if (subPlan >= 0) {
+            sb.append(".");
+            sb.append(subPlan);
+            sb.append(".");
+            sb.append(subPos);
+            sb.append(" - Subplan ");
+        } else {
+            sb.append(" - MicroOp ");
+        }
+        sb.append(name);
+        if (input >= 0) {
+            sb.append(" input [");
+            sb.append(input);
+            sb.append("] ");
+        }
+        return sb.toString();
+    }
+
+    private static String makeId(String base, int id, int subPlan, int subPos) {
+        return base + "." + id + (subPlan >= 0 ? "." + subPlan : "") + (subPos >= 0 ? "." + subPos : "");
+    }
+
+    private static IOperatorStats makeStatForRuntimeFact(IPushRuntimeFactory factory, String base, String baseId,
+            int pos, int subPlan, int subPos) {
+        return new OperatorStats(makeStatName(base, factory.toString(), pos, -1, subPlan, subPos),
+                makeId(baseId, pos, subPlan, subPos));
+    }
+
+    public static Map<IPushRuntimeFactory, IOperatorStats> makeMicroOpStats(AlgebricksPipeline pipe,
+            IOperatorStats outerStats) {
+        Map<IPushRuntimeFactory, IOperatorStats> microOpStats = new HashMap<>();
+        String baseName = outerStats.getName().split(" - ")[0];
+        String baseId = outerStats.getOperatorId();
+        List<SubplanRuntimeFactory> subplans = new ArrayList<>();
+        for (int i = 0; i < pipe.getRuntimeFactories().length; i++) {
+            IPushRuntimeFactory fact = pipe.getRuntimeFactories()[i];
+            //TODO: don't use instanceof
+            if (fact instanceof SubplanRuntimeFactory) {
+                SubplanRuntimeFactory subplanFact = (SubplanRuntimeFactory) fact;
+                subplans.add(subplanFact);
+                List<AlgebricksPipeline> pipelines = subplanFact.getPipelines();
+                for (AlgebricksPipeline p : pipelines) {
+                    IPushRuntimeFactory[] subplanFactories = p.getRuntimeFactories();
+                    for (int j = subplanFactories.length - 1; j > 0; j--) {
+                        microOpStats.put(subplanFactories[j], makeStatForRuntimeFact(subplanFactories[j], baseName,
+                                baseId, i, pipelines.indexOf(p), j));
+                    }
+                }
+            }
+            microOpStats.put(fact, makeStatForRuntimeFact(fact, baseName, baseId, i, -1, -1));
+        }
+        for (SubplanRuntimeFactory sub : subplans) {
+            sub.setStats(microOpStats);
+        }
+        return microOpStats;
+    }
+
     private class SourcePushRuntime extends AbstractUnaryOutputSourceOperatorNodePushable {
         private final IHyracksTaskContext ctx;
 
@@ -99,7 +169,7 @@
                     outputArity > 0 ? AlgebricksMetaOperatorDescriptor.this.outRecDescs[0] : null;
             PipelineAssembler pa =
                     new PipelineAssembler(pipeline, inputArity, outputArity, null, pipelineOutputRecordDescriptor);
-            startOfPipeline = pa.assemblePipeline(writer, ctx);
+            startOfPipeline = pa.assemblePipeline(writer, ctx, new HashMap<>());
             HyracksDataException exception = null;
             try {
                 startOfPipeline.open();
@@ -126,16 +196,18 @@
         public String getDisplayName() {
             return "Empty Tuple Source";
         }
+
     }
 
     private IOperatorNodePushable createOneInputOneOutputPushRuntime(final IHyracksTaskContext ctx,
             final IRecordDescriptorProvider recordDescProvider) {
-        return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+        return new AbstractUnaryInputUnaryOutputIntrospectingOperatorNodePushable() {
 
             private IFrameWriter startOfPipeline;
             private boolean opened = false;
+            private IOperatorStats parentStats = NoOpOperatorStats.INSTANCE;
+            private Map<IPushRuntimeFactory, IOperatorStats> microOpStats = new HashMap<>();
 
-            @Override
             public void open() throws HyracksDataException {
                 if (startOfPipeline == null) {
                     RecordDescriptor pipelineOutputRecordDescriptor =
@@ -144,7 +216,7 @@
                             .getInputRecordDescriptor(AlgebricksMetaOperatorDescriptor.this.getActivityId(), 0);
                     PipelineAssembler pa = new PipelineAssembler(pipeline, inputArity, outputArity,
                             pipelineInputRecordDescriptor, pipelineOutputRecordDescriptor);
-                    startOfPipeline = pa.assemblePipeline(writer, ctx);
+                    startOfPipeline = pa.assemblePipeline(writer, ctx, microOpStats);
                 }
                 opened = true;
                 startOfPipeline.open();
@@ -175,9 +247,38 @@
             }
 
             @Override
+            public void deinitialize() throws HyracksDataException {
+                super.deinitialize();
+            }
+
+            @Override
             public String toString() {
                 return AlgebricksMetaOperatorDescriptor.this.toString();
             }
+
+            @Override
+            public void addStats(IOperatorStats stats) throws HyracksDataException {
+                microOpStats = makeMicroOpStats(pipeline, stats);
+                for (IOperatorStats stat : microOpStats.values()) {
+                    ctx.getStatsCollector().add(stat);
+                }
+            }
+
+            @Override
+            public void setUpstreamStats(IOperatorStats stats) {
+                parentStats = stats;
+            }
+
+            @Override
+            public long getTotalTime() {
+                return startOfPipeline instanceof ITimedWriter ? ((ITimedWriter) startOfPipeline).getTotalTime() : 0;
+            }
+
+            @Override
+            public IOperatorStats getStats() {
+                IPushRuntimeFactory[] facts = pipeline.getRuntimeFactories();
+                return microOpStats.getOrDefault(facts[facts.length - 1], NoOpOperatorStats.INSTANCE);
+            }
         };
     }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
index 858fcfa..202c087 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
@@ -25,6 +25,8 @@
 import org.apache.hyracks.algebricks.runtime.base.EnforcePushRuntime;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.base.ProfiledPushRuntime;
+import org.apache.hyracks.algebricks.runtime.operators.std.EmptyTupleSourceRuntimeFactory;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.EnforceFrameWriter;
@@ -32,6 +34,7 @@
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.JobFlag;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 
 public class PipelineAssembler {
 
@@ -55,6 +58,11 @@
     }
 
     public IFrameWriter assemblePipeline(IFrameWriter writer, IHyracksTaskContext ctx) throws HyracksDataException {
+        return assemblePipeline(writer, ctx, new HashMap<>());
+    }
+
+    public IFrameWriter assemblePipeline(IFrameWriter writer, IHyracksTaskContext ctx,
+            Map<IPushRuntimeFactory, IOperatorStats> microOpStats) throws HyracksDataException {
         // should enforce protocol
         boolean enforce = ctx.getJobFlags().contains(JobFlag.ENFORCE_CONTRACT);
         boolean profile = ctx.getJobFlags().contains(JobFlag.PROFILE_RUNTIME);
@@ -67,7 +75,21 @@
             IPushRuntimeFactory runtimeFactory = runtimeFactories[i];
             IPushRuntime[] newRuntimes = runtimeFactory.createPushRuntime(ctx);
             for (int j = 0; j < newRuntimes.length; j++) {
-                if (enforce) {
+                //ETS is wrapped externally, and doesn't need the micro-op wrapper since it isn't a pipeline
+                //we also want to avoid any instances of NoOp stats in the pipeline that snuck in somehow
+                boolean shouldProfile = profile && !(runtimeFactory instanceof EmptyTupleSourceRuntimeFactory)
+                        && microOpStats.containsKey(runtimeFactory);
+                if (shouldProfile) {
+                    ProfiledPushRuntime profiled;
+                    if (j == 0) {
+                        profiled = (ProfiledPushRuntime) ProfiledPushRuntime.time(newRuntimes[j],
+                                microOpStats.get(runtimeFactory));
+                    } else {
+                        profiled = (ProfiledPushRuntime) ProfiledPushRuntime.time(newRuntimes[j],
+                                microOpStats.get(runtimeFactory), false);
+                    }
+                    newRuntimes[j] = profiled;
+                } else if (enforce && !profile) {
                     newRuntimes[j] = EnforcePushRuntime.enforce(newRuntimes[j]);
                 }
                 if (i == runtimeFactories.length - 1) {
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
index 349e50f..7feca3c 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
@@ -20,7 +20,9 @@
 
 import java.io.DataOutput;
 import java.nio.ByteBuffer;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
@@ -32,16 +34,19 @@
 import org.apache.hyracks.algebricks.runtime.operators.std.NestedTupleSourceRuntimeFactory.NestedTupleSourceRuntime;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.ProfiledFrameWriter;
 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.JobFlag;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 
 public class SubplanRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
 
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 3L;
 
     private final List<AlgebricksPipeline> pipelines;
 
@@ -51,6 +56,8 @@
 
     private final IMissingWriterFactory[] missingWriterFactories;
 
+    private final Map<IPushRuntimeFactory, IOperatorStats> stats;
+
     public SubplanRuntimeFactory(List<AlgebricksPipeline> pipelines, IMissingWriterFactory[] missingWriterFactories,
             RecordDescriptor inputRecordDesc, RecordDescriptor outputRecordDesc, int[] projectionList) {
         super(projectionList);
@@ -61,6 +68,7 @@
         if (projectionList != null) {
             throw new NotImplementedException();
         }
+        this.stats = new HashMap<>();
     }
 
     @Override
@@ -78,6 +86,14 @@
         return sb.toString();
     }
 
+    public List<AlgebricksPipeline> getPipelines() {
+        return pipelines;
+    }
+
+    public void setStats(Map<IPushRuntimeFactory, IOperatorStats> stats) {
+        this.stats.putAll(stats);
+    }
+
     @Override
     public AbstractOneInputOneOutputPushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx)
             throws HyracksDataException {
@@ -92,8 +108,11 @@
 
         boolean first;
 
+        boolean profile;
+
         SubplanPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
             this.ctx = ctx;
+            this.profile = ctx.getJobFlags().contains(JobFlag.PROFILE_RUNTIME);
             this.first = true;
 
             IMissingWriter[] missingWriters = new IMissingWriter[missingWriterFactories.length];
@@ -114,6 +133,11 @@
                 if (i == 0) {
                     // primary pipeline
                     outputWriter = new TupleOuterProduct(pipelineLastRecordDescriptor, missingWriters);
+                    //this is necessary to track the output of the last operator to the outer product,
+                    //i.e. the last real operator in pipeline 0 of the subplan
+                    if (profile) {
+                        outputWriter = new ProfiledFrameWriter(outputWriter);
+                    }
                     outputRecordDescriptor = SubplanRuntimeFactory.this.outputRecordDesc;
                 } else {
                     // secondary pipeline
@@ -127,7 +151,8 @@
                 }
 
                 PipelineAssembler pa = new PipelineAssembler(pipeline, 1, 1, inputRecordDesc, outputRecordDescriptor);
-                startOfPipelines[i] = (NestedTupleSourceRuntime) pa.assemblePipeline(outputWriter, ctx);
+                IFrameWriter head = pa.assemblePipeline(outputWriter, ctx, stats);
+                startOfPipelines[i] = (NestedTupleSourceRuntime) head;
                 pipelineAssemblers[i] = pa;
             }
         }
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/operators/std/SwitchOperatorDescriptor.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SwitchOperatorDescriptor.java
new file mode 100644
index 0000000..0fbb1d5
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/SwitchOperatorDescriptor.java
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.algebricks.runtime.operators.std;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+import org.apache.commons.lang3.mutable.MutableBoolean;
+import org.apache.hyracks.algebricks.data.IBinaryIntegerInspector;
+import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
+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.evaluators.EvaluatorContext;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.ActivityId;
+import org.apache.hyracks.api.dataflow.IActivityGraphBuilder;
+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.util.CleanupUtils;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import org.apache.hyracks.dataflow.std.base.AbstractActivityNode;
+import org.apache.hyracks.dataflow.std.base.AbstractReplicateOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputOperatorNodePushable;
+
+/**
+ * The switch operator propagates each tuple of the input to one or more output branches based on a given output mapping.
+ * For each tuple, we peek at the value of a given field f. We look up the value of the field in the supplied
+ * output mapping and propagate the tuple to all corresponding output branches.
+ */
+public class SwitchOperatorDescriptor extends AbstractReplicateOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+    final static int SWITCHER_ACTIVITY_ID = 0;
+
+    private final IScalarEvaluatorFactory branchingExprEvalFactory;
+    private final IBinaryIntegerInspectorFactory intInspectorFactory;
+    private final Map<Integer, int[]> outputMapping;
+
+    /**
+     * @param spec
+     * @param rDesc
+     * @param outputArity equal to the number of non-materialized outputs
+     * @param branchingExprEvalFactory containing the index of the relevant field f
+     * @param intInspectorFactory
+     * @param outputMapping the supplied mapping from field values to arrays of output branch numbers
+     */
+    public SwitchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor rDesc, int outputArity,
+            IScalarEvaluatorFactory branchingExprEvalFactory, IBinaryIntegerInspectorFactory intInspectorFactory,
+            Map<Integer, int[]> outputMapping) {
+        super(spec, rDesc, outputArity);
+        if (outputArity != numberOfNonMaterializedOutputs) {
+            throw new IllegalArgumentException();
+        }
+        this.branchingExprEvalFactory = branchingExprEvalFactory;
+        this.intInspectorFactory = intInspectorFactory;
+        this.outputMapping = outputMapping;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        SwitcherActivityNode sma = new SwitcherActivityNode(new ActivityId(odId, SWITCHER_ACTIVITY_ID));
+        builder.addActivity(this, sma);
+        builder.addSourceEdge(0, sma, 0);
+        for (int i = 0; i < outputArity; i++) {
+            builder.addTargetEdge(i, sma, i);
+        }
+    }
+
+    private final class SwitcherActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public SwitcherActivityNode(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+                throws HyracksDataException {
+            final IFrameWriter[] writers = new IFrameWriter[outputArity];
+            final boolean[] isOpen = new boolean[outputArity];
+            final IPointable p = VoidPointable.FACTORY.createPointable();
+            // To deal with each tuple in a frame
+            final FrameTupleAccessor accessor = new FrameTupleAccessor(outRecDescs[0]);
+            final FrameTupleAppender[] appenders = new FrameTupleAppender[outputArity];
+            final FrameTupleReference tRef = new FrameTupleReference();
+            final IBinaryIntegerInspector intInspector = intInspectorFactory.createBinaryIntegerInspector(ctx);
+            final IEvaluatorContext evalCtx = new EvaluatorContext(ctx);
+            final IScalarEvaluator eval = branchingExprEvalFactory.createScalarEvaluator(evalCtx);
+            final MutableBoolean hasFailed = new MutableBoolean(false);
+            for (int i = 0; i < outputArity; i++) {
+                appenders[i] = new FrameTupleAppender(new VSizeFrame(ctx), true);
+            }
+
+            return new AbstractUnaryInputOperatorNodePushable() {
+                @Override
+                public void open() throws HyracksDataException {
+                    for (int i = 0; i < outputArity; i++) {
+                        isOpen[i] = true;
+                        writers[i].open();
+                    }
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer bufferAccessor) throws HyracksDataException {
+                    // Tuple based access
+                    accessor.reset(bufferAccessor);
+                    int tupleCount = accessor.getTupleCount();
+
+                    for (int i = 0; i < tupleCount; i++) {
+                        // Get the value of the relevant field in the given tuple.
+                        tRef.reset(accessor, i);
+                        eval.evaluate(tRef, p);
+                        int fieldValue =
+                                intInspector.getIntegerValue(p.getByteArray(), p.getStartOffset(), p.getLength());
+
+                        // Look up the corresponding output branches based on the given mapping
+                        int[] outputBranches = outputMapping.get(fieldValue);
+
+                        // Propagate to each output branch
+                        for (int j : outputBranches) {
+                            FrameUtils.appendToWriter(writers[j], appenders[j], accessor, i);
+                        }
+                    }
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    Throwable hde = null;
+                    // write if hasn't failed
+                    if (!hasFailed.booleanValue()) {
+                        for (int i = 0; i < outputArity; i++) {
+                            if (isOpen[i]) {
+                                try {
+                                    appenders[i].write(writers[i], true);
+                                } catch (Throwable th) {
+                                    hde = th;
+                                    break;
+                                }
+                            }
+                        }
+                    }
+
+                    // fail the writers
+                    if (hde != null) {
+                        for (int i = 0; i < outputArity; i++) {
+                            if (isOpen[i]) {
+                                CleanupUtils.fail(writers[i], hde);
+                            }
+                        }
+                    }
+
+                    // close
+                    for (int i = 0; i < outputArity; i++) {
+                        if (isOpen[i]) {
+                            hde = CleanupUtils.close(writers[i], hde);
+                        }
+                    }
+                    if (hde != null) {
+                        throw HyracksDataException.create(hde);
+                    }
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                    hasFailed.setTrue();
+                    HyracksDataException hde = null;
+                    for (int i = 0; i < outputArity; i++) {
+                        if (isOpen[i]) {
+                            try {
+                                writers[i].fail();
+                            } catch (Throwable th) {
+                                if (hde == null) {
+                                    hde = HyracksDataException.create(th);
+                                } else {
+                                    hde.addSuppressed(th);
+                                }
+                            }
+                        }
+                    }
+                    if (hde != null) {
+                        throw hde;
+                    }
+                }
+
+                @Override
+                public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                    writers[index] = writer;
+                }
+
+                @Override
+                public void flush() throws HyracksDataException {
+                    for (int i = 0; i < outputArity; i++) {
+                        if (isOpen[i]) {
+                            appenders[i].flush(writers[i]);
+                        }
+                    }
+                }
+            };
+        }
+    }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2-switch-0.tbl b/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2-switch-0.tbl
new file mode 100644
index 0000000..026bbdb
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2-switch-0.tbl
@@ -0,0 +1,6 @@
+1,second branch1
+2,third branch1
+1,second branch2
+2,third branch2
+1,second branch3
+2,third branch3
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2-switch-1.tbl b/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2-switch-1.tbl
new file mode 100644
index 0000000..464a6448
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2-switch-1.tbl
@@ -0,0 +1,7 @@
+0,first branch1
+2,third branch1
+0,first branch2
+2,third branch2
+0,first branch3
+2,third branch3
+0,first branch4
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2-switch-2.tbl b/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2-switch-2.tbl
new file mode 100644
index 0000000..b9eabbc
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2-switch-2.tbl
@@ -0,0 +1,7 @@
+0,first branch1
+1,second branch1
+0,first branch2
+1,second branch2
+0,first branch3
+1,second branch3
+0,first branch4
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2.tbl b/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2.tbl
new file mode 100644
index 0000000..373cd0f
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-tests/data/device0/data/simple/int-string-part2.tbl
@@ -0,0 +1,10 @@
+0|first branch1
+1|second branch1
+2|third branch1
+0|first branch2
+1|second branch2
+2|third branch2
+0|first branch3
+1|second branch3
+2|third branch3
+0|first branch4
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/pom.xml b/hyracks-fullstack/algebricks/algebricks-tests/pom.xml
index 34894cb..4f563a6 100644
--- a/hyracks-fullstack/algebricks/algebricks-tests/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-tests/pom.xml
@@ -117,16 +117,19 @@
       <!-- Dependency management inherited from top-level hyracks -->
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-control-cc</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-control-nc</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
@@ -142,6 +145,7 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-control-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
@@ -152,11 +156,13 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>algebricks-data</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>algebricks-core</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
@@ -167,28 +173,34 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-dataflow-std</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-util</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-ipc</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-databind</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>commons-io</groupId>
       <artifactId>commons-io</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
+      <scope>test</scope>
     </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java b/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
index aeb22b6..d23f7f9 100644
--- a/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
+++ b/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
@@ -23,6 +23,7 @@
 import java.io.FileReader;
 import java.io.IOException;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.io.FileUtils;
@@ -58,6 +59,7 @@
 import org.apache.hyracks.algebricks.runtime.operators.std.StreamProjectRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.StreamSelectRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.StringStreamingRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.std.SwitchOperatorDescriptor;
 import org.apache.hyracks.algebricks.runtime.operators.std.UnnestRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
 import org.apache.hyracks.algebricks.tests.util.AlgebricksHyracksIntegrationUtil;
@@ -698,6 +700,76 @@
     }
 
     @Test
+    public void scanSwitchWrite() throws Exception {
+        final int outputArity = 3;
+
+        JobSpecification spec = new JobSpecification(FRAME_SIZE);
+
+        String inputFileName[] = { "data" + File.separator + "simple" + File.separator + "int-string-part2.tbl",
+                "data" + File.separator + "simple" + File.separator + "int-string-part2-switch-0.tbl",
+                "data" + File.separator + "simple" + File.separator + "int-string-part2-switch-1.tbl",
+                "data" + File.separator + "simple" + File.separator + "int-string-part2-switch-2.tbl" };
+        File[] inputFiles = new File[inputFileName.length];
+        for (int i = 0; i < inputFileName.length; i++) {
+            inputFiles[i] = new File(inputFileName[i]);
+        }
+        File[] outputFile = new File[outputArity];
+        FileSplit[] outputFileSplit = new FileSplit[outputArity];
+        for (int i = 0; i < outputArity; i++) {
+            outputFileSplit[i] = createFile(AlgebricksHyracksIntegrationUtil.nc1);
+            outputFile[i] = outputFileSplit[i].getFile(AlgebricksHyracksIntegrationUtil.nc1.getIoManager());
+        }
+
+        FileSplit[] inputSplits =
+                new FileSplit[] { new ManagedFileSplit(AlgebricksHyracksIntegrationUtil.NC1_ID, inputFileName[0]) };
+        IFileSplitProvider intSplitProvider = new ConstantFileSplitProvider(inputSplits);
+
+        RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                IntegerSerializerDeserializer.INSTANCE, new UTF8StringSerializerDeserializer() });
+
+        IValueParserFactory[] valueParsers =
+                new IValueParserFactory[] { IntegerParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE };
+
+        FileScanOperatorDescriptor intScanner = new FileScanOperatorDescriptor(spec, intSplitProvider,
+                new DelimitedDataTupleParserFactory(valueParsers, '|'), scannerDesc);
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, intScanner, DEFAULT_NODES);
+
+        HashMap<Integer, int[]> outputMapping = new HashMap<>();
+        outputMapping.put(0, new int[] { 1, 2 });
+        outputMapping.put(1, new int[] { 0, 2 });
+        outputMapping.put(2, new int[] { 0, 1 });
+
+        SwitchOperatorDescriptor switchOp = new SwitchOperatorDescriptor(spec, scannerDesc, outputArity,
+                new TupleFieldEvaluatorFactory(0), BinaryIntegerInspectorImpl.FACTORY, outputMapping);
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, switchOp,
+                new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+
+        IOperatorDescriptor outputOp[] = new IOperatorDescriptor[outputFile.length];
+        for (int i = 0; i < outputArity; i++) {
+            outputOp[i] = new LineFileWriteOperatorDescriptor(spec, new FileSplit[] { outputFileSplit[i] });
+            PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, outputOp[i],
+                    new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
+        }
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), intScanner, 0, switchOp, 0);
+        for (int i = 0; i < outputArity; i++) {
+            spec.connect(new OneToOneConnectorDescriptor(spec), switchOp, i, outputOp[i], 0);
+        }
+
+        for (int i = 0; i < outputArity; i++) {
+            spec.addRoot(outputOp[i]);
+        }
+        AlgebricksHyracksIntegrationUtil.runJob(spec);
+
+        for (int i = 0; i < outputArity; i++) {
+            compareFiles("data" + File.separator + "device0" + File.separator + inputFileName[i + 1],
+                    outputFile[i].getAbsolutePath());
+        }
+    }
+
+    @Test
     public void scanMicroSortWrite() throws Exception {
         JobSpecification spec = new JobSpecification(FRAME_SIZE);
 
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/IServiceContext.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/IServiceContext.java
index 0a99c45..7145f50 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/IServiceContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/IServiceContext.java
@@ -19,11 +19,13 @@
 package org.apache.hyracks.api.application;
 
 import java.io.Serializable;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ThreadFactory;
 
 import org.apache.hyracks.api.config.IApplicationConfig;
+import org.apache.hyracks.api.deployment.DeploymentId;
 import org.apache.hyracks.api.io.IPersistedResourceRegistry;
-import org.apache.hyracks.api.job.IJobSerializerDeserializerContainer;
+import org.apache.hyracks.api.job.IJobSerializerDeserializer;
 import org.apache.hyracks.api.messages.IMessageBroker;
 import org.apache.hyracks.api.service.IControllerService;
 
@@ -41,7 +43,7 @@
 
     IMessageBroker getMessageBroker();
 
-    IJobSerializerDeserializerContainer getJobSerializerDeserializerContainer();
+    ConcurrentMap<DeploymentId, IJobSerializerDeserializer> getJobSerializerDeserializerContainer();
 
     ThreadFactory getThreadFactory();
 
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/NetworkAddress.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/NetworkAddress.java
index 75fbb92..9f9186b 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/NetworkAddress.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/NetworkAddress.java
@@ -35,8 +35,6 @@
     private String address;
     // Cached locally, not serialized
     private volatile byte[] ipAddress;
-    // Cached locally, not serialized
-    private volatile InetSocketAddress inetSocketAddress;
 
     private int port;
 
@@ -76,11 +74,12 @@
         return ipAddress;
     }
 
-    public InetSocketAddress resolveInetSocketAddress() {
-        if (inetSocketAddress == null) {
-            inetSocketAddress = new InetSocketAddress(address, port);
+    public InetSocketAddress toResolvedInetSocketAddress() throws UnknownHostException {
+        InetSocketAddress addr = toInetSocketAddress();
+        if (addr.isUnresolved()) {
+            throw new UnknownHostException(getAddress());
         }
-        return inetSocketAddress;
+        return addr;
     }
 
     public InetSocketAddress toInetSocketAddress() {
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IIntrospectingOperator.java
similarity index 68%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IIntrospectingOperator.java
index 6afbccb..64897ef 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ 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
@@ -18,15 +19,8 @@
  */
 package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 
-    void pause();
-
-    void resume();
+public interface IIntrospectingOperator {
+    void setOperatorStats(IOperatorStats stats);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ISelfProfilingNodePushable.java
similarity index 68%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ISelfProfilingNodePushable.java
index 6afbccb..7402179 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ISelfProfilingNodePushable.java
@@ -16,17 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 
-    void pause();
+public interface ISelfProfilingNodePushable extends IStatsContainingNodePushable {
+    void addStats(IOperatorStats stats) throws HyracksDataException;
 
-    void resume();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IStatsContainingNodePushable.java
similarity index 68%
rename from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
rename to hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IStatsContainingNodePushable.java
index 6afbccb..96a3ae9 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IStatsContainingNodePushable.java
@@ -18,15 +18,8 @@
  */
 package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 
-    void pause();
-
-    void resume();
+public interface IStatsContainingNodePushable {
+    IOperatorStats getStats();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ITimedWriter.java
similarity index 68%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ITimedWriter.java
index 6afbccb..7b0f8c8 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ITimedWriter.java
@@ -18,15 +18,11 @@
  */
 package org.apache.hyracks.api.dataflow;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 
-    void pause();
+public interface ITimedWriter extends IFrameWriter {
+    void setUpstreamStats(IOperatorStats stats);
 
-    void resume();
+    long getTotalTime();
 }
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..cfb0e7b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledFrameWriter.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 static org.apache.hyracks.api.job.profiling.NoOpOperatorStats.INVALID_ODID;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.com.job.profiling.counters.Counter;
+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.NoOpOperatorStats;
+import org.apache.hyracks.api.job.profiling.OperatorStats;
+import org.apache.hyracks.api.job.profiling.counters.ICounter;
+import org.apache.hyracks.api.util.HyracksRunnable;
+import org.apache.hyracks.api.util.HyracksThrowingConsumer;
+import org.apache.hyracks.util.IntSerDeUtils;
+
+public class ProfiledFrameWriter implements ITimedWriter {
+
+    // The downstream data consumer of this writer.
+    private final IFrameWriter writer;
+    protected IOperatorStats upstreamStats = NoOpOperatorStats.INSTANCE;
+    private int minSz = Integer.MAX_VALUE;
+    private int maxSz = -1;
+    private long avgSz;
+    private ICounter totalTime;
+
+    public ProfiledFrameWriter(IFrameWriter writer) {
+        this.writer = writer;
+        this.totalTime = new Counter("totalTime");
+    }
+
+    @Override
+    public void setUpstreamStats(IOperatorStats stats) {
+        this.upstreamStats = stats;
+    }
+
+    public static void timeMethod(HyracksRunnable r, ICounter c) throws HyracksDataException {
+        long nt = 0;
+        try {
+            nt = System.nanoTime();
+            r.run();
+        } finally {
+            c.update(System.nanoTime() - nt);
+        }
+    }
+
+    private void timeMethod(HyracksThrowingConsumer<ByteBuffer> c, ByteBuffer buffer) throws HyracksDataException {
+        long nt = 0;
+        try {
+            nt = System.nanoTime();
+            c.accept(buffer);
+        } finally {
+            totalTime.update(System.nanoTime() - nt);
+        }
+    }
+
+    @Override
+    public final void open() throws HyracksDataException {
+        timeMethod(writer::open, totalTime);
+    }
+
+    private void updateTupleStats(ByteBuffer buffer) {
+        int tupleCountOffset = FrameHelper.getTupleCountOffset(buffer.limit());
+        int tupleCount = IntSerDeUtils.getInt(buffer.array(), tupleCountOffset);
+        ICounter tupleCounter = upstreamStats.getTupleCounter();
+        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++;
+        }
+        upstreamStats.getMaxTupleSz().set(maxSz);
+        upstreamStats.getMinTupleSz().set(minSz);
+        upstreamStats.getAverageTupleSz().set(avgSz);
+        tupleCounter.update(tupleCount);
+    }
+
+    @Override
+    public final void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        updateTupleStats(buffer);
+        timeMethod(writer::nextFrame, buffer);
+    }
+
+    @Override
+    public final void flush() throws HyracksDataException {
+        timeMethod(writer::flush, totalTime);
+    }
+
+    @Override
+    public final void fail() throws HyracksDataException {
+        timeMethod(writer::fail, totalTime);
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        timeMethod(writer::close, totalTime);
+    }
+
+    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, INVALID_ODID);
+            statsCollector.add(stats);
+            return new ProfiledFrameWriter(writer);
+        } else {
+            return writer;
+        }
+    }
+
+    @Override
+    public long getTotalTime() {
+        return totalTime.get();
+    }
+
+}
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..1984d8e
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/ProfiledOperatorNodePushable.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 java.util.Map;
+
+import org.apache.hyracks.api.com.job.profiling.counters.Counter;
+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.job.profiling.counters.ICounter;
+import org.apache.hyracks.api.rewriter.runtime.SuperActivityOperatorNodePushable;
+
+public class ProfiledOperatorNodePushable implements IOperatorNodePushable, IStatsContainingNodePushable {
+
+    private final IOperatorNodePushable op;
+    private final Map<Integer, ITimedWriter> inputs;
+    private final Map<Integer, ITimedWriter> outputs;
+    private final IOperatorStats stats;
+    private final ICounter totalTime;
+
+    ProfiledOperatorNodePushable(IOperatorNodePushable op, IOperatorStats stats) {
+        this.stats = stats;
+        this.op = op;
+        inputs = new HashMap<>();
+        outputs = new HashMap<>();
+        this.totalTime = new Counter("totalTime");
+    }
+
+    @Override
+    public void initialize() throws HyracksDataException {
+        ProfiledFrameWriter.timeMethod(op::initialize, totalTime);
+    }
+
+    @Override
+    public void deinitialize() throws HyracksDataException {
+        long ownTime = totalTime.get();
+        for (ITimedWriter i : inputs.values()) {
+            ownTime += i.getTotalTime();
+        }
+        for (ITimedWriter w : outputs.values()) {
+            ownTime -= w.getTotalTime();
+        }
+        op.deinitialize();
+        stats.getTimeCounter().set(ownTime);
+    }
+
+    @Override
+    public int getInputArity() {
+        return op.getInputArity();
+    }
+
+    @Override
+    public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc)
+            throws HyracksDataException {
+        if (writer instanceof ITimedWriter) {
+            ITimedWriter wrapper = (ITimedWriter) writer;
+            if (op instanceof ISelfProfilingNodePushable) {
+                wrapper.setUpstreamStats(((ISelfProfilingNodePushable) op).getStats());
+            } else {
+                wrapper.setUpstreamStats(stats);
+            }
+            outputs.put(index, wrapper);
+        }
+        op.setOutputFrameWriter(index, writer, recordDesc);
+    }
+
+    @Override
+    public IFrameWriter getInputFrameWriter(int index) {
+        if (inputs.get(index) == null) {
+            ProfiledFrameWriter pfw = new ProfiledFrameWriter(op.getInputFrameWriter(index));
+            inputs.put(index, pfw);
+            return pfw;
+        } else {
+            return inputs.get(index);
+        }
+    }
+
+    @Override
+    public String getDisplayName() {
+        return op.getDisplayName();
+    }
+
+    public IOperatorStats getStats() {
+        return stats;
+    }
+
+    public static IOperatorNodePushable time(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().toString());
+        if (!(op instanceof ISelfProfilingNodePushable)) {
+            statsCollector.add(stats);
+        }
+        if (op instanceof IIntrospectingOperator) {
+            ((IIntrospectingOperator) op).setOperatorStats(stats);
+        }
+        if (op instanceof ISelfProfilingNodePushable) {
+            ((ISelfProfilingNodePushable) op).addStats(stats);
+        }
+        if (!(op instanceof ProfiledOperatorNodePushable) && !(op instanceof SuperActivityOperatorNodePushable)) {
+            return new ProfiledOperatorNodePushable(op, stats);
+        }
+        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().toString());
+        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/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/IJobSerializerDeserializerContainer.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/IJobSerializerDeserializerContainer.java
deleted file mode 100644
index e63c18b..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/IJobSerializerDeserializerContainer.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.hyracks.api.job;
-
-import org.apache.hyracks.api.deployment.DeploymentId;
-
-public interface IJobSerializerDeserializerContainer {
-
-    /**
-     * Get the IJobSerializerDeserializer implementation instance for a specific deployment id
-     *
-     * @param deploymentId
-     * @return
-     */
-    public IJobSerializerDeserializer getJobSerializerDeserializer(DeploymentId deploymentId);
-
-    /**
-     * Add a deployment with the job serializer deserializer
-     *
-     * @param deploymentId
-     * @param jobSerDe
-     */
-    public void addJobSerializerDeserializer(DeploymentId deploymentId, IJobSerializerDeserializer jobSerDe);
-
-    /**
-     * Remove a deployment
-     *
-     * @param deploymentId
-     */
-    public void removeJobSerializerDeserializer(DeploymentId deploymentId);
-
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSerializerDeserializerContainer.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSerializerDeserializerContainer.java
deleted file mode 100644
index 7f3194e..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSerializerDeserializerContainer.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.hyracks.api.job;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.hyracks.api.deployment.DeploymentId;
-
-public class JobSerializerDeserializerContainer implements IJobSerializerDeserializerContainer {
-
-    private IJobSerializerDeserializer defaultJobSerDe = new JobSerializerDeserializer();
-    private Map<DeploymentId, IJobSerializerDeserializer> jobSerializerDeserializerMap =
-            new ConcurrentHashMap<DeploymentId, IJobSerializerDeserializer>();
-
-    @Override
-    public synchronized IJobSerializerDeserializer getJobSerializerDeserializer(DeploymentId deploymentId) {
-        if (deploymentId == null) {
-            return defaultJobSerDe;
-        }
-        IJobSerializerDeserializer jobSerDe = jobSerializerDeserializerMap.get(deploymentId);
-        return jobSerDe;
-    }
-
-    @Override
-    public synchronized void addJobSerializerDeserializer(DeploymentId deploymentId,
-            IJobSerializerDeserializer jobSerDe) {
-        jobSerializerDeserializerMap.put(deploymentId, jobSerDe);
-    }
-
-    @Override
-    public synchronized void removeJobSerializerDeserializer(DeploymentId deploymentId) {
-        jobSerializerDeserializerMap.remove(deploymentId);
-    }
-
-    @Override
-    public String toString() {
-        return jobSerializerDeserializerMap.toString();
-    }
-
-}
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 0d38fac..7770c4f 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
@@ -18,6 +18,8 @@
  */
 package org.apache.hyracks.api.job.profiling;
 
+import java.io.DataInput;
+import java.io.IOException;
 import java.io.Serializable;
 import java.util.Map;
 
@@ -44,13 +46,63 @@
     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();
+
+    String getOperatorId();
 
     void updateIndexesStats(Map<String, IndexStats> indexesStats);
 
     Map<String, IndexStats> getIndexesStats();
 
     void updateFrom(IOperatorStats stats);
+
+    static IOperatorStats create(DataInput input) throws IOException {
+        String name = input.readUTF();
+        if (NoOpOperatorStats.NOOP_NAME.equals(name)) {
+            return NoOpOperatorStats.INSTANCE;
+        }
+        String operatorId = input.readUTF();
+        OperatorStats operatorStats = new OperatorStats(name, operatorId);
+        operatorStats.readFields(input);
+        return operatorStats;
+    }
 }
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 c903c68..3169c81 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
@@ -21,7 +21,6 @@
 import java.io.Serializable;
 import java.util.Map;
 
-import org.apache.hyracks.api.dataflow.IPassableTimer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IWritable;
 
@@ -38,14 +37,7 @@
     /**
      * @param operatorName
      * @return {@link IOperatorStats} for the operator with name <code>operatorName</code>
-     * if it already exists, and adds it if it does not.
-     */
-    IOperatorStats getOrAddOperatorStats(String operatorName);
-
-    /**
-     * @param operatorName
-     * @return {@link IOperatorStats} for the operator with name <code>operatorName</code>
-     * if it already exists, or {@code null} if the operator does not exist
+     * if one exists or else null.
      */
     IOperatorStats getOperatorStats(String operatorName);
 
@@ -61,18 +53,4 @@
      */
     IOperatorStats getAggregatedStats();
 
-    /**
-     * Pause an operator's timer, to pass it to another operator
-     * @param newHolder the timer that is starting execution
-     * @return the current nanoTime when the clock was taken from the other operator
-     */
-    long takeClock(IPassableTimer newHolder);
-
-    /**
-     * Resume an operator's timer, when a downstream operator has finished execution of
-     * the method the upstream operator called
-     * @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..d427d14
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/NoOpOperatorStats.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 java.util.Collections;
+import java.util.Map;
+
+import org.apache.hyracks.api.job.profiling.counters.ICounter;
+
+public class NoOpOperatorStats implements IOperatorStats {
+
+    private static final long serialVersionUID = 9055940222300360256L;
+
+    public static final NoOpOperatorStats INSTANCE = new NoOpOperatorStats();
+    public static final String INVALID_ODID = "ODID:-1";
+    public static final String NOOP_NAME = "NoOp";
+
+    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;
+        }
+    };
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+        output.writeUTF(NOOP_NAME);
+        output.writeUTF(INVALID_ODID);
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        // nothing
+    }
+
+    @Override
+    public String getName() {
+        return NOOP_NAME;
+    }
+
+    @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 String getOperatorId() {
+        return INVALID_ODID;
+    }
+
+    @Override
+    public void updateIndexesStats(Map<String, IndexStats> indexesStats) {
+        // no op
+    }
+
+    @Override
+    public Map<String, IndexStats> getIndexesStats() {
+        return Collections.emptyMap();
+    }
+
+    @Override
+    public void updateFrom(IOperatorStats stats) {
+        // no op
+    }
+}
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 a3a5073..412b788 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
@@ -28,32 +28,46 @@
 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 = 6401830963361567126L;
     public final String operatorName;
+
+    public final String operatorId;
     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;
     private final Map<String, IndexStats> indexesStats;
 
-    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, String operatorId) {
         if (operatorName == null || operatorName.isEmpty()) {
             throw new IllegalArgumentException("operatorName must not be null or empty");
         }
         this.operatorName = operatorName;
+        this.operatorId = operatorId;
         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);
         indexesStats = new HashMap<>();
     }
 
-    public static IOperatorStats create(DataInput input) throws IOException {
-        String name = input.readUTF();
-        OperatorStats operatorStats = new OperatorStats(name);
-        operatorStats.readFields(input);
-        return operatorStats;
-    }
-
     @Override
     public String getName() {
         return operatorName;
@@ -70,8 +84,53 @@
     }
 
     @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 String getOperatorId() {
+        return operatorId;
     }
 
     @Override
@@ -100,16 +159,25 @@
     public void updateFrom(IOperatorStats stats) {
         tupleCounter.update(stats.getTupleCounter().get());
         timeCounter.update(stats.getTimeCounter().get());
-        diskIoCounter.update(stats.getDiskIoCounter().get());
+        pageReads.update(stats.getPageReads().get());
         updateIndexesStats(stats.getIndexesStats());
     }
 
     @Override
     public void writeFields(DataOutput output) throws IOException {
         output.writeUTF(operatorName);
+        output.writeUTF(operatorId);
         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());
         writeIndexesStats(output);
     }
 
@@ -117,7 +185,15 @@
     public void readFields(DataInput input) throws IOException {
         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());
         readIndexesStats(input);
     }
 
@@ -140,8 +216,13 @@
 
     @Override
     public String toString() {
-        return "{ " + "\"operatorName\": \"" + operatorName + "\", " + "\"" + tupleCounter.getName() + "\": "
-                + tupleCounter.get() + ", \"" + timeCounter.getName() + "\": " + timeCounter.get() + ", \""
-                + ", \"indexStats\": \"" + indexesStats + "\" }";
+        return "{ " + "\"operatorName\": \"" + operatorName + "\", " + "\"id\": \"" + operatorId + "\", " + "\""
+                + tupleCounter.getName() + "\": " + 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() + ", \"indexStats\": \"" + indexesStats + "\" }";
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/lifecycle/LifeCycleComponentManager.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/lifecycle/LifeCycleComponentManager.java
index a9b7a97..54bac09 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/lifecycle/LifeCycleComponentManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/lifecycle/LifeCycleComponentManager.java
@@ -18,7 +18,7 @@
  */
 package org.apache.hyracks.api.lifecycle;
 
-import static org.apache.hyracks.util.ExitUtil.EC_UNHANDLED_EXCEPTION;
+import static org.apache.hyracks.util.ExitUtil.EC_HALT_UNHANDLED_EXCEPTION;
 
 import java.io.File;
 import java.io.FileOutputStream;
@@ -57,9 +57,9 @@
     @Override
     public void uncaughtException(Thread t, Throwable e) {
         try {
-            LOGGER.log(Level.ERROR, "Uncaught Exception from thread " + t.getName() + ". Calling shutdown hook", e);
+            LOGGER.fatal("Uncaught Exception from thread {}; halting", t.getName(), e);
         } finally {
-            ExitUtil.exit(EC_UNHANDLED_EXCEPTION);
+            ExitUtil.halt(EC_HALT_UNHANDLED_EXCEPTION);
         }
     }
 
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 10af2fd..17f5cb1 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
@@ -45,7 +45,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,12 +109,14 @@
          */
         Set<Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>>> sources = new HashSet<>();
         for (Entry<ActivityId, IActivity> entry : startActivities.entrySet()) {
-            IOperatorNodePushable opPushable = null;
+            IOperatorNodePushable opPushable;
             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());
             } else {
                 opPushable = entry.getValue().createPushRuntime(ctx, recordDescProvider, partition, nPartitions);
+                ProfiledOperatorNodePushable.onlyAddStats(opPushable, ctx, entry.getKey());
             }
             operatorNodePushablesBFSOrder.add(opPushable);
             operatorNodePushables.put(entry.getKey(), opPushable);
@@ -143,11 +145,13 @@
             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);
+                    destOp = ProfiledOperatorNodePushable.time(wrapped, ctx, destId);
                 } 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/CleanupUtils.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/CleanupUtils.java
index 220311e..a29a04a 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/CleanupUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/CleanupUtils.java
@@ -67,7 +67,7 @@
             } catch (Throwable th) { // NOSONAR Will be suppressed
                 try {
                     LOGGER.log(ExceptionUtils.causedByInterrupt(th) ? Level.DEBUG : Level.WARN,
-                            "Failure closing a closeable resource of class {}", writer.getClass().getSimpleName(), th);
+                            "Failure closing a closeable resource of class {}", writer.getClass().getName(), th);
                 } catch (Throwable loggingFailure) { // NOSONAR: Ignore catching Throwable
                     // NOSONAR: Ignore logging failure
                 }
@@ -92,7 +92,7 @@
         } catch (Throwable th) { // NOSONAR Will be suppressed
             try {
                 LOGGER.log(ExceptionUtils.causedByInterrupt(th) ? Level.DEBUG : Level.WARN,
-                        "Failure failing " + writer.getClass().getSimpleName(), th);
+                        "Failure failing " + writer.getClass().getName(), th);
             } catch (Throwable loggingFailure) { // NOSONAR: Ignore catching Throwable
                 // NOSONAR ignore logging failure
             }
@@ -119,7 +119,7 @@
             } catch (Throwable th) { // NOSONAR Will be suppressed
                 try {
                     LOGGER.log(ExceptionUtils.causedByInterrupt(th) ? Level.DEBUG : Level.WARN,
-                            "Failure closing a closeable resource", th);
+                            "Failure closing a closeable resource {}", closable.getClass().getName(), th);
                 } catch (Throwable loggingFailure) { // NOSONAR: Ignore catching Throwable
                     // NOSONAR ignore logging failure
                 }
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-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksRunnable.java
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksRunnable.java
index 6afbccb..a82c150 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksRunnable.java
@@ -16,17 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
+package org.apache.hyracks.api.util;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-    void pause();
-
-    void resume();
+@FunctionalInterface
+public interface HyracksRunnable {
+    void run() throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
index 4fc503d..a50cc31 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
@@ -248,12 +248,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);
             }
@@ -502,19 +503,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 {
@@ -606,10 +603,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-api/src/main/resources/errormsg/en.properties b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
index 7db5d49..c46558a 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
@@ -121,7 +121,7 @@
 101 = Page %1$s does not exist in file %2$s
 102 = Failed to open virtual buffer cache since it is already open
 103 = Failed to close virtual buffer cache since it is already closed
-104 = Index does not exist
+104 = Index does not exist (%1$s)
 105 = Cannot drop in-use index (%1$s)
 106 = Failed to deactivate the bloom filter since it is pinned by other users
 107 = The given search predicate can't be null.
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/string/UTF8StringUtilTest.java b/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/string/UTF8StringUtilTest.java
index 6f3782b..37ab002 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/string/UTF8StringUtilTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/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;
@@ -79,13 +80,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;
         }
@@ -101,7 +103,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);
 
@@ -119,7 +121,6 @@
                 assertEquals(str1.compareToIgnoreCase(str2), lowerCaseCompareTo(buffer1, 0, buffer2, 0));
                 break;
         }
-
     }
 
     @Test
diff --git a/hyracks-fullstack/hyracks/hyracks-client/pom.xml b/hyracks-fullstack/hyracks/hyracks-client/pom.xml
index 9d4ac27..3f1e7cf 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-client/pom.xml
@@ -89,11 +89,13 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-control-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-control-cc</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSetReader.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSetReader.java
index b29e2ea2..992c8be 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSetReader.java
+++ b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSetReader.java
@@ -18,8 +18,6 @@
  */
 package org.apache.hyracks.client.result;
 
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
@@ -28,7 +26,6 @@
 import org.apache.hyracks.api.channels.IInputChannelMonitor;
 import org.apache.hyracks.api.comm.FrameHelper;
 import org.apache.hyracks.api.comm.IFrame;
-import org.apache.hyracks.api.comm.NetworkAddress;
 import org.apache.hyracks.api.context.IHyracksCommonContext;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -146,8 +143,7 @@
 
     private SocketAddress getSocketAddress(ResultDirectoryRecord record) throws HyracksDataException {
         try {
-            final NetworkAddress netAddr = record.getNetworkAddress();
-            return new InetSocketAddress(InetAddress.getByAddress(netAddr.lookupIpAddress()), netAddr.getPort());
+            return record.getNetworkAddress().toResolvedInetSocketAddress();
         } catch (UnknownHostException e) {
             throw HyracksDataException.create(e);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
index 71f7b44..71139f9 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
@@ -26,11 +26,30 @@
     <artifactId>hyracks-control</artifactId>
     <version>0.3.8.2-SNAPSHOT</version>
   </parent>
-
   <properties>
     <root.dir>${basedir}/../../..</root.dir>
     <sonar.sources>pom.xml,src/main/java,src/main/resources</sonar.sources>
   </properties>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <configuration>
+          <ignoredUsedUndeclaredDependencies combine.children="append">
+            <ignoredUsedUndeclaredDependency>io.netty:netty-transport:*</ignoredUsedUndeclaredDependency>
+          </ignoredUsedUndeclaredDependencies>
+        </configuration>
+        <executions>
+          <execution>
+            <phase>process-test-classes</phase>
+            <goals>
+              <goal>analyze-only</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>  </plugins>
+  </build>
   <dependencies>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
index f11e7ff..d6698fe 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
@@ -35,6 +35,7 @@
 import java.util.TreeMap;
 import java.util.concurrent.ExecutorService;
 
+import org.apache.hyracks.api.application.IApplication;
 import org.apache.hyracks.api.application.ICCApplication;
 import org.apache.hyracks.api.client.ClusterControllerInfo;
 import org.apache.hyracks.api.comm.NetworkAddress;
@@ -49,7 +50,6 @@
 import org.apache.hyracks.api.job.JobIdFactory;
 import org.apache.hyracks.api.job.JobParameterByteStore;
 import org.apache.hyracks.api.job.resource.IJobCapacityController;
-import org.apache.hyracks.api.network.INetworkSecurityConfig;
 import org.apache.hyracks.api.network.INetworkSecurityManager;
 import org.apache.hyracks.api.service.IControllerService;
 import org.apache.hyracks.api.topology.ClusterTopology;
@@ -170,8 +170,7 @@
         File jobLogFolder = new File(ccConfig.getRootDir(), "logs/jobs");
         jobLog = new LogFile(jobLogFolder);
 
-        final INetworkSecurityConfig securityConfig = getNetworkSecurityConfig();
-        networkSecurityManager = new NetworkSecurityManager(securityConfig);
+        networkSecurityManager = createNetworkSecurityManager(ccConfig.getAppConfig(), application);
 
         // WorkQueue is in charge of heartbeat as well as other events.
         workQueue = new WorkQueue("ClusterController", Thread.MAX_PRIORITY);
@@ -567,8 +566,9 @@
         return networkSecurityManager;
     }
 
-    protected INetworkSecurityConfig getNetworkSecurityConfig() {
-        return NetworkSecurityConfig.of(ccConfig.isSslEnabled(), ccConfig.getKeyStorePath(),
-                ccConfig.getKeyStorePassword(), ccConfig.getTrustStorePath());
+    protected INetworkSecurityManager createNetworkSecurityManager(IApplicationConfig appConfig, IApplication app)
+            throws Exception {
+        return new NetworkSecurityManager(NetworkSecurityConfig.of(ccConfig.isSslEnabled(), ccConfig.getKeyStorePath(),
+                ccConfig.getKeyStorePassword(), ccConfig.getTrustStorePath()));
     }
 }
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/ApplicationMessageWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ApplicationMessageWork.java
index 771832e..6630ba7 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ApplicationMessageWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ApplicationMessageWork.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.api.messages.IMessage;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.common.deployment.DeploymentUtils;
+import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -34,8 +35,8 @@
 public class ApplicationMessageWork extends AbstractHeartbeatWork {
 
     private static final Logger LOGGER = LogManager.getLogger();
-    private byte[] message;
-    private DeploymentId deploymentId;
+    private final byte[] message;
+    private final DeploymentId deploymentId;
 
     public ApplicationMessageWork(ClusterControllerService ccs, byte[] message, DeploymentId deploymentId,
             String nodeId) {
@@ -57,6 +58,11 @@
     }
 
     @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
+
+    @Override
     public String toString() {
         return getName() + ": nodeID: " + nodeId;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java
index c36b887..f08e209 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java
@@ -24,10 +24,12 @@
 import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.common.work.AbstractWork;
 import org.apache.hyracks.control.common.work.IResultCallback;
+import org.apache.logging.log4j.Level;
 
 public class GetNodeControllersInfoWork extends AbstractWork {
+
     private final INodeManager nodeManager;
-    private IResultCallback<Map<String, NodeControllerInfo>> callback;
+    private final IResultCallback<Map<String, NodeControllerInfo>> callback;
 
     public GetNodeControllersInfoWork(INodeManager nodeManager,
             IResultCallback<Map<String, NodeControllerInfo>> callback) {
@@ -39,4 +41,9 @@
     public void run() {
         callback.setValue(nodeManager.getNodeControllerInfoMap());
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultDirectoryAddressWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultDirectoryAddressWork.java
index bf95ff2..b7dbd75 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultDirectoryAddressWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetResultDirectoryAddressWork.java
@@ -22,8 +22,10 @@
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.common.work.IResultCallback;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
+import org.apache.logging.log4j.Level;
 
 public class GetResultDirectoryAddressWork extends SynchronizableWork {
+
     private final ClusterControllerService ccs;
 
     private final IResultCallback<NetworkAddress> callback;
@@ -42,4 +44,9 @@
             callback.setException(e);
         }
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java
index 77d2f82..6262c47 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java
@@ -32,13 +32,14 @@
 import org.apache.logging.log4j.Logger;
 
 public class JobCleanupWork extends AbstractWork {
+
     private static final Logger LOGGER = LogManager.getLogger();
 
-    private IJobManager jobManager;
-    private JobId jobId;
-    private JobStatus status;
-    private List<Exception> exceptions;
-    private IResultCallback<Void> callback;
+    private final IJobManager jobManager;
+    private final JobId jobId;
+    private final JobStatus status;
+    private final List<Exception> exceptions;
+    private final IResultCallback<Void> callback;
 
     public JobCleanupWork(IJobManager jobManager, JobId jobId, JobStatus status, List<Exception> exceptions,
             IResultCallback<Void> callback) {
@@ -51,12 +52,10 @@
 
     @Override
     public void run() {
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("Cleanup for job: {}", jobId);
-        }
+        LOGGER.info("cleaning up {} on NCs, status={}", jobId, status);
         final JobRun jobRun = jobManager.get(jobId);
         if (jobRun == null) {
-            LOGGER.debug("Ignoring cleanup for unknown job: {}", jobId);
+            LOGGER.debug("ignoring cleanup for unknown {}", jobId);
             return;
         }
         try {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobStartWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobStartWork.java
index cfedfc9..86e36d6 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobStartWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobStartWork.java
@@ -35,8 +35,13 @@
 import org.apache.hyracks.control.common.deployment.DeploymentUtils;
 import org.apache.hyracks.control.common.work.IResultCallback;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 public class JobStartWork extends SynchronizableWork {
+
+    private static final Logger LOGGER = LogManager.getLogger();
     private final ClusterControllerService ccs;
     private final byte[] acggfBytes;
     private final Set<JobFlag> jobFlags;
@@ -67,6 +72,7 @@
             JobId jobId;
             JobRun run;
             jobId = jobIdFactory.create();
+            LOGGER.debug("created {}", jobId);
             if (deployedJobSpecId == null) {
                 //Need to create the ActivityClusterGraph
                 IActivityClusterGraphGeneratorFactory acggf = (IActivityClusterGraphGeneratorFactory) DeploymentUtils
@@ -85,4 +91,9 @@
             callback.setException(e);
         }
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java
index c3a09f9..76a72c0 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java
@@ -35,6 +35,7 @@
 import org.apache.logging.log4j.Logger;
 
 public class JobletCleanupNotificationWork extends AbstractHeartbeatWork {
+
     private static final Logger LOGGER = LogManager.getLogger();
 
     private final JobId jobId;
@@ -46,6 +47,7 @@
 
     @Override
     public void runWork() {
+        LOGGER.debug("node {} finished job clean-up {}", nodeId, jobId);
         IJobManager jobManager = ccs.getJobManager();
         final JobRun run = jobManager.get(jobId);
         if (run == null) {
@@ -82,4 +84,9 @@
     public String toString() {
         return getName() + " jobId:" + jobId + ", nodeId:" + nodeId;
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
index ec21785..810fda2 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
@@ -31,6 +31,7 @@
 import org.apache.hyracks.control.common.controllers.NodeRegistration;
 import org.apache.hyracks.control.common.ipc.NodeControllerRemoteProxy;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
+import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -83,4 +84,9 @@
             nc.sendRegistrationResult(params, e);
         }
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RemoveDeadNodesWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RemoveDeadNodesWork.java
index ee10669..cd9b6d0 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RemoveDeadNodesWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RemoveDeadNodesWork.java
@@ -50,9 +50,7 @@
             Collection<JobId> affectedJobIds = result.getRight();
             int size = affectedJobIds.size();
             if (size > 0) {
-                if (LOGGER.isInfoEnabled()) {
-                    LOGGER.info("Number of affected jobs: " + size);
-                }
+                LOGGER.info("number of affected jobs due to dead nodes removal {}", size);
                 IJobManager jobManager = ccs.getJobManager();
                 for (JobId jobId : affectedJobIds) {
                     JobRun run = jobManager.get(jobId);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java
index 0c53142..869caa5 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java
@@ -29,8 +29,13 @@
 import org.apache.hyracks.control.common.job.profiling.om.JobProfile;
 import org.apache.hyracks.control.common.job.profiling.om.JobletProfile;
 import org.apache.hyracks.control.common.job.profiling.om.TaskProfile;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 public class TaskCompleteWork extends AbstractTaskLifecycleWork {
+
+    private static final Logger LOGGER = LogManager.getLogger();
     private final TaskProfile statistics;
 
     public TaskCompleteWork(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId,
@@ -44,8 +49,10 @@
         IJobManager jobManager = ccs.getJobManager();
         JobRun run = jobManager.get(jobId);
         if (run == null) {
+            LOGGER.debug("node completed task for unknown job {}:{}:{}", nodeId, jobId, taId);
             return;
         }
+        LOGGER.debug("node completed task {}:{}:{}", nodeId, jobId, taId);
         if (statistics != null) {
             JobProfile jobProfile = run.getJobProfile();
             Map<String, JobletProfile> jobletProfiles = jobProfile.getJobletProfiles();
@@ -63,4 +70,9 @@
     public String toString() {
         return getName() + ": [" + nodeId + "[" + jobId + ":" + taId + "]";
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ServiceContext.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ServiceContext.java
index b6d32a9..411b134 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ServiceContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/application/ServiceContext.java
@@ -19,14 +19,16 @@
 package org.apache.hyracks.control.common.application;
 
 import java.io.Serializable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ThreadFactory;
 
 import org.apache.hyracks.api.application.IServerContext;
 import org.apache.hyracks.api.application.IServiceContext;
 import org.apache.hyracks.api.config.IApplicationConfig;
+import org.apache.hyracks.api.deployment.DeploymentId;
 import org.apache.hyracks.api.io.IPersistedResourceRegistry;
-import org.apache.hyracks.api.job.IJobSerializerDeserializerContainer;
-import org.apache.hyracks.api.job.JobSerializerDeserializerContainer;
+import org.apache.hyracks.api.job.IJobSerializerDeserializer;
 import org.apache.hyracks.api.messages.IMessageBroker;
 
 public abstract class ServiceContext implements IServiceContext {
@@ -35,7 +37,7 @@
     protected ThreadFactory threadFactory;
     protected Serializable distributedState;
     protected IMessageBroker messageBroker;
-    protected IJobSerializerDeserializerContainer jobSerDeContainer = new JobSerializerDeserializerContainer();
+    protected ConcurrentMap<DeploymentId, IJobSerializerDeserializer> jobSerDeContainer = new ConcurrentHashMap<>();
     protected IPersistedResourceRegistry persistedResourceRegistry;
 
     public ServiceContext(IServerContext serverCtx, IApplicationConfig appConfig, ThreadFactory threadFactory) {
@@ -60,7 +62,7 @@
     }
 
     @Override
-    public IJobSerializerDeserializerContainer getJobSerializerDeserializerContainer() {
+    public ConcurrentMap<DeploymentId, IJobSerializerDeserializer> getJobSerializerDeserializerContainer() {
         return this.jobSerDeContainer;
     }
 
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/deployment/DeploymentUtils.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/deployment/DeploymentUtils.java
index ceb0da8..0b33d1c 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/deployment/DeploymentUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/deployment/DeploymentUtils.java
@@ -26,6 +26,7 @@
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.ConcurrentMap;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.http.HttpEntity;
@@ -37,7 +38,6 @@
 import org.apache.hyracks.api.deployment.DeploymentId;
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.job.IJobSerializerDeserializer;
-import org.apache.hyracks.api.job.IJobSerializerDeserializerContainer;
 import org.apache.hyracks.api.util.JavaSerializationUtils;
 import org.apache.hyracks.control.common.context.ServerContext;
 import org.apache.logging.log4j.LogManager;
@@ -62,9 +62,10 @@
      * @param ctx
      * @throws HyracksException
      */
-    public static void undeploy(DeploymentId deploymentId, IJobSerializerDeserializerContainer container,
-            ServerContext ctx) throws HyracksException {
-        container.removeJobSerializerDeserializer(deploymentId);
+    public static void undeploy(DeploymentId deploymentId,
+            ConcurrentMap<DeploymentId, IJobSerializerDeserializer> container, ServerContext ctx)
+            throws HyracksException {
+        container.remove(deploymentId);
         String rootDir = ctx.getBaseDir().toString();
         String deploymentDir = rootDir.endsWith(File.separator) ? rootDir + DEPLOYMENT + File.separator + deploymentId
                 : rootDir + File.separator + DEPLOYMENT + File.separator + deploymentId;
@@ -93,13 +94,11 @@
      *            true is NC/false is CC
      * @throws HyracksException
      */
-    public static void deploy(DeploymentId deploymentId, List<URL> urls, IJobSerializerDeserializerContainer container,
-            ServerContext ctx, boolean isNC) throws HyracksException {
-        IJobSerializerDeserializer jobSerDe = container.getJobSerializerDeserializer(deploymentId);
-        if (jobSerDe == null) {
-            jobSerDe = new ClassLoaderJobSerializerDeserializer();
-            container.addJobSerializerDeserializer(deploymentId, jobSerDe);
-        }
+    public static void deploy(DeploymentId deploymentId, List<URL> urls,
+            ConcurrentMap<DeploymentId, IJobSerializerDeserializer> container, ServerContext ctx, boolean isNC)
+            throws HyracksException {
+        IJobSerializerDeserializer jobSerDe =
+                container.computeIfAbsent(deploymentId, d -> new ClassLoaderJobSerializerDeserializer());
         String rootDir = ctx.getBaseDir().toString();
         String deploymentDir = rootDir.endsWith(File.separator) ? rootDir + DEPLOYMENT + File.separator + deploymentId
                 : rootDir + File.separator + DEPLOYMENT + File.separator + deploymentId;
@@ -120,9 +119,8 @@
     public static Object deserialize(byte[] bytes, DeploymentId deploymentId, IServiceContext serviceCtx)
             throws HyracksException {
         try {
-            IJobSerializerDeserializerContainer jobSerDeContainer = serviceCtx.getJobSerializerDeserializerContainer();
             IJobSerializerDeserializer jobSerDe =
-                    deploymentId == null ? null : jobSerDeContainer.getJobSerializerDeserializer(deploymentId);
+                    deploymentId == null ? null : serviceCtx.getJobSerializerDeserializerContainer().get(deploymentId);
             return jobSerDe == null ? JavaSerializationUtils.deserialize(bytes) : jobSerDe.deserialize(bytes);
         } catch (Exception e) {
             throw HyracksException.create(e);
@@ -141,9 +139,8 @@
     public static Class<?> loadClass(String className, DeploymentId deploymentId, IServiceContext serviceCtx)
             throws HyracksException {
         try {
-            IJobSerializerDeserializerContainer jobSerDeContainer = serviceCtx.getJobSerializerDeserializerContainer();
             IJobSerializerDeserializer jobSerDe =
-                    deploymentId == null ? null : jobSerDeContainer.getJobSerializerDeserializer(deploymentId);
+                    deploymentId == null ? null : serviceCtx.getJobSerializerDeserializerContainer().get(deploymentId);
             return jobSerDe == null ? JavaSerializationUtils.loadClass(className) : jobSerDe.loadClass(className);
         } catch (ClassNotFoundException | IOException e) {
             throw HyracksException.create(e);
@@ -160,9 +157,8 @@
      */
     public static ClassLoader getClassLoader(DeploymentId deploymentId, IServiceContext appCtx)
             throws HyracksException {
-        IJobSerializerDeserializerContainer jobSerDeContainer = appCtx.getJobSerializerDeserializerContainer();
         IJobSerializerDeserializer jobSerDe =
-                deploymentId == null ? null : jobSerDeContainer.getJobSerializerDeserializer(deploymentId);
+                deploymentId == null ? null : appCtx.getJobSerializerDeserializerContainer().get(deploymentId);
         return jobSerDe == null ? DeploymentUtils.class.getClassLoader() : jobSerDe.getClassLoader();
     }
 
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 580a560..7cfbda0 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
@@ -18,25 +18,24 @@
  */
 package org.apache.hyracks.control.common.job.profiling;
 
+import static org.apache.hyracks.api.job.profiling.NoOpOperatorStats.INVALID_ODID;
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.ArrayDeque;
 import java.util.Collections;
-import java.util.Deque;
 import java.util.LinkedHashMap;
 import java.util.Map;
 
-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 = 6858817639895434379L;
 
     private final Map<String, IOperatorStats> operatorStatsMap = new LinkedHashMap<>();
-    private transient Deque<IPassableTimer> clockHolder = new ArrayDeque<>();
 
     @Override
     public void add(IOperatorStats operatorStats) {
@@ -47,13 +46,8 @@
     }
 
     @Override
-    public IOperatorStats getOrAddOperatorStats(String operatorName) {
-        return operatorStatsMap.computeIfAbsent(operatorName, OperatorStats::new);
-    }
-
-    @Override
     public IOperatorStats getOperatorStats(String operatorName) {
-        return operatorStatsMap.get(operatorName);
+        return operatorStatsMap.getOrDefault(operatorName, NoOpOperatorStats.INSTANCE);
     }
 
     @Override
@@ -69,11 +63,12 @@
 
     @Override
     public IOperatorStats getAggregatedStats() {
-        IOperatorStats aggregatedStats = new OperatorStats("aggregated");
+        IOperatorStats aggregatedStats = new OperatorStats("aggregated", INVALID_ODID);
         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());
+            aggregatedStats.coldReadCounter().update(stats.coldReadCounter().get());
         }
         return aggregatedStats;
     }
@@ -81,8 +76,8 @@
     @Override
     public void writeFields(DataOutput output) throws IOException {
         output.writeInt(operatorStatsMap.size());
-        for (IOperatorStats operatorStats : operatorStatsMap.values()) {
-            operatorStats.writeFields(output);
+        for (IOperatorStats stats : operatorStatsMap.values()) {
+            stats.writeFields(output);
         }
     }
 
@@ -90,28 +85,9 @@
     public void readFields(DataInput input) throws IOException {
         int operatorCount = input.readInt();
         for (int i = 0; i < operatorCount; i++) {
-            IOperatorStats opStats = OperatorStats.create(input);
+            IOperatorStats opStats = IOperatorStats.create(input);
             operatorStatsMap.put(opStats.getName(), opStats);
         }
     }
 
-    @Override
-    public long takeClock(IPassableTimer newHolder) {
-        if (newHolder != null) {
-            if (clockHolder.peek() != null) {
-                clockHolder.peek().pause();
-            }
-            clockHolder.push(newHolder);
-        }
-        return System.nanoTime();
-    }
-
-    @Override
-    public void giveClock(IPassableTimer currHolder) {
-        clockHolder.removeLastOccurrence(currHolder);
-        if (clockHolder.peek() != null) {
-            clockHolder.peek().resume();
-        }
-    }
-
 }
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 ee49908..40bc1ba 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
@@ -18,6 +18,8 @@
  */
 package org.apache.hyracks.control.common.job.profiling.om;
 
+import static org.apache.hyracks.api.job.profiling.NoOpOperatorStats.INVALID_ODID;
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -26,12 +28,14 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.concurrent.TimeUnit;
 
 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;
@@ -42,6 +46,10 @@
     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;
 
@@ -68,12 +76,36 @@
         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;
+    }
+
+    public long getQueueWaitTimeInNanos() {
+        return TimeUnit.MILLISECONDS.toNanos(startTime - createTime);
+    }
+
     @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()) {
@@ -98,6 +130,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++) {
@@ -110,6 +146,10 @@
     @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());
@@ -141,12 +181,12 @@
             for (int i = 0; i < n; i++) {
                 String operatorName = operatorNames.get(i);
                 IOperatorStats opTaskStats = statsCollector.getOperatorStats(operatorName);
-                if (opTaskStats == null) {
+                if (opTaskStats.equals(NoOpOperatorStats.INSTANCE)) {
                     continue;
                 }
                 IOperatorStats opOutStats = outStats[i];
                 if (opOutStats == null) {
-                    opOutStats = new OperatorStats(operatorName);
+                    opOutStats = new OperatorStats(operatorName, INVALID_ODID);
                     outStats[i] = opOutStats;
                 }
                 opOutStats.updateFrom(opTaskStats);
@@ -154,4 +194,5 @@
         }
         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..84376f6 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
@@ -18,6 +18,8 @@
  */
 package org.apache.hyracks.control.common.job.profiling.om;
 
+import static org.apache.hyracks.api.job.profiling.NoOpOperatorStats.INVALID_ODID;
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -128,9 +130,30 @@
         opTimes.forEach((key, value) -> {
             ObjectNode jpe = om.createObjectNode();
             jpe.put("name", key);
-            jpe.put("time", Double
+            jpe.put("run-time", Double
                     .parseDouble(new DecimalFormat("#.####").format((double) value.getTimeCounter().get() / 1000000)));
-            jpe.put("disk-io", value.getDiskIoCounter().get());
+            if (!value.getOperatorId().equals(INVALID_ODID)) {
+                jpe.put("runtime-id", value.getOperatorId());
+            }
+            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/NodeControllerService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
index f69d106..e173dcb 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
@@ -45,6 +45,7 @@
 import org.apache.hyracks.api.client.NodeControllerInfo;
 import org.apache.hyracks.api.client.NodeStatus;
 import org.apache.hyracks.api.comm.NetworkAddress;
+import org.apache.hyracks.api.config.IApplicationConfig;
 import org.apache.hyracks.api.control.CcId;
 import org.apache.hyracks.api.deployment.DeploymentId;
 import org.apache.hyracks.api.exceptions.ErrorCode;
@@ -56,7 +57,6 @@
 import org.apache.hyracks.api.job.JobParameterByteStore;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponentManager;
 import org.apache.hyracks.api.lifecycle.LifeCycleComponentManager;
-import org.apache.hyracks.api.network.INetworkSecurityConfig;
 import org.apache.hyracks.api.network.INetworkSecurityManager;
 import org.apache.hyracks.api.result.IResultPartitionManager;
 import org.apache.hyracks.api.service.IControllerService;
@@ -197,8 +197,7 @@
         if (application == null) {
             throw new IllegalArgumentException("INCApplication cannot be null");
         }
-        final INetworkSecurityConfig securityConfig = getNetworkSecurityConfig();
-        networkSecurityManager = new NetworkSecurityManager(securityConfig);
+        networkSecurityManager = createNetworkSecurityManager(ncConfig.getAppConfig(), application);
         this.application = application;
         id = ncConfig.getNodeId();
         if (id == null) {
@@ -726,8 +725,9 @@
         return networkSecurityManager;
     }
 
-    protected INetworkSecurityConfig getNetworkSecurityConfig() {
-        return NetworkSecurityConfig.of(ncConfig.isSslEnabled(), ncConfig.getKeyStorePath(),
-                ncConfig.getKeyStorePassword(), ncConfig.getTrustStorePath());
+    protected INetworkSecurityManager createNetworkSecurityManager(IApplicationConfig appConfig,
+            INCApplication application) {
+        return new NetworkSecurityManager(NetworkSecurityConfig.of(ncConfig.isSslEnabled(), ncConfig.getKeyStorePath(),
+                ncConfig.getKeyStorePassword(), ncConfig.getTrustStorePath()));
     }
 }
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 900ac7e..0c5c233 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
@@ -515,9 +515,6 @@
 
     @Override
     public synchronized void subscribeThreadToStats(IThreadStatsCollector threadStatsCollector) {
-        if (!isRuntimeProfilingEnabled()) {
-            return;
-        }
         synchronized (threadStatsCollectors) {
             threadStatsCollectors.add(threadStatsCollector);
             final long threadId = Thread.currentThread().getId();
@@ -528,9 +525,6 @@
 
     @Override
     public synchronized void unsubscribeThreadFromStats() {
-        if (!isRuntimeProfilingEnabled()) {
-            return;
-        }
         synchronized (threadStatsCollectors) {
             threadStatsCollectors.forEach(IThreadStatsCollector::unsubscribe);
         }
@@ -572,8 +566,4 @@
         return "{ \"class\" : \"" + getClass().getSimpleName() + "\", \"node\" : \"" + ncs.getId() + "\" \"jobId\" : \""
                 + joblet.getJobId() + "\", \"taskId\" : \"" + taskAttemptId + "\" }";
     }
-
-    private boolean isRuntimeProfilingEnabled() {
-        return getJobFlags().contains(JobFlag.PROFILE_RUNTIME);
-    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/profiling/IOCounterProc.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/profiling/IOCounterProc.java
index 8882271..fe84ee7 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/profiling/IOCounterProc.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/profiling/IOCounterProc.java
@@ -50,9 +50,7 @@
     public long getWrites() {
         try {
             List<String> rows = getInfo();
-            long writes = extractRow(rows, 5);
-            long cancelledWrites = extractRow(rows, 6);
-            return writes - cancelledWrites;
+            return extractRow(rows, 5);
         } catch (Exception e) {
             LOGGER.log(failureCount++ > 0 ? Level.DEBUG : Level.WARN, "Failure getting writes", e);
             return IOCounterDefault.IO_COUNTER_UNAVAILABLE;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
index 6d4f173..f386a89 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
@@ -29,11 +29,12 @@
 import org.apache.logging.log4j.Logger;
 
 public class ApplicationMessageWork extends AbstractWork {
+
     private static final Logger LOGGER = LogManager.getLogger();
-    private byte[] message;
-    private DeploymentId deploymentId;
-    private String nodeId;
-    private NodeControllerService ncs;
+    private final byte[] message;
+    private final DeploymentId deploymentId;
+    private final String nodeId;
+    private final NodeControllerService ncs;
 
     public ApplicationMessageWork(NodeControllerService ncs, byte[] message, DeploymentId deploymentId, String nodeId) {
         this.ncs = ncs;
@@ -59,6 +60,11 @@
     }
 
     @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
+
+    @Override
     public String toString() {
         return getName() + ": nodeId: " + nodeId;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/CleanupJobletWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/CleanupJobletWork.java
index 75edd38..efc8467 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/CleanupJobletWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/CleanupJobletWork.java
@@ -25,10 +25,12 @@
 import org.apache.hyracks.control.common.work.AbstractWork;
 import org.apache.hyracks.control.nc.Joblet;
 import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 public class CleanupJobletWork extends AbstractWork {
+
     private static final Logger LOGGER = LogManager.getLogger();
 
     private final NodeControllerService ncs;
@@ -45,7 +47,7 @@
 
     @Override
     public void run() {
-        LOGGER.debug("cleaning up after job: {}", jobId);
+        LOGGER.debug("cleaning up {}", jobId);
         ncs.removeJobParameterByteStore(jobId);
         ncs.getPartitionManager().jobCompleted(jobId, status);
         Map<JobId, Joblet> jobletMap = ncs.getJobletMap();
@@ -59,4 +61,9 @@
     public String toString() {
         return getName() + " jobId:" + jobId + ", status:" + status;
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskCompleteWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskCompleteWork.java
index 60860c5..7e1b6f5 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskCompleteWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/NotifyTaskCompleteWork.java
@@ -18,6 +18,8 @@
  */
 package org.apache.hyracks.control.nc.work;
 
+import org.apache.hyracks.api.dataflow.TaskAttemptId;
+import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.control.common.job.profiling.om.TaskProfile;
 import org.apache.hyracks.control.common.work.AbstractWork;
 import org.apache.hyracks.control.nc.NodeControllerService;
@@ -38,20 +40,27 @@
 
     @Override
     public void run() {
-        TaskProfile taskProfile = new TaskProfile(task.getTaskAttemptId(), task.getPartitionSendProfile(),
+        JobId jobId = task.getJoblet().getJobId();
+        TaskAttemptId taskAttemptId = task.getTaskAttemptId();
+        LOGGER.debug("notifying CC of task complete {}:{}", jobId, taskAttemptId);
+        TaskProfile taskProfile = new TaskProfile(taskAttemptId, task.getPartitionSendProfile(),
                 task.getStatsCollector(), task.getWarnings(), task.getWarningCollector().getTotalWarningsCount());
         try {
-            ncs.getClusterController(task.getJobletContext().getJobId().getCcId()).notifyTaskComplete(
-                    task.getJobletContext().getJobId(), task.getTaskAttemptId(), ncs.getId(), taskProfile);
+            ncs.getClusterController(task.getJobletContext().getJobId().getCcId())
+                    .notifyTaskComplete(task.getJobletContext().getJobId(), taskAttemptId, ncs.getId(), taskProfile);
         } catch (Exception e) {
-            LOGGER.log(Level.ERROR, "Failed notifying task complete for " + task.getTaskAttemptId(), e);
+            LOGGER.log(Level.ERROR, "Failed notifying task complete for {}", taskAttemptId, e);
         }
         task.getJoblet().removeTask(task);
     }
 
     @Override
     public String toString() {
-        return getName() + ": [" + ncs.getId() + "[" + task.getJoblet().getJobId() + ":" + task.getTaskAttemptId()
-                + "]";
+        return getName() + ": [" + task.getJoblet().getJobId() + ":" + task.getTaskAttemptId() + "]";
+    }
+
+    @Override
+    public Level logLevel() {
+        return Level.TRACE;
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
index cfd69ce..a6952b9 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
@@ -18,8 +18,6 @@
  */
 package org.apache.hyracks.control.nc.work;
 
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
 import java.util.Map;
 
 import org.apache.hyracks.api.comm.NetworkAddress;
@@ -50,13 +48,8 @@
             Map<JobId, Joblet> jobletMap = ncs.getJobletMap();
             Joblet ji = jobletMap.get(pid.getJobId());
             if (ji != null) {
-                PartitionChannel channel =
-                        new PartitionChannel(pid,
-                                new NetworkInputChannel(ncs.getNetworkManager(),
-                                        new InetSocketAddress(
-                                                InetAddress.getByAddress(networkAddress.lookupIpAddress()),
-                                                networkAddress.getPort()),
-                                        pid, 5));
+                PartitionChannel channel = new PartitionChannel(pid, new NetworkInputChannel(ncs.getNetworkManager(),
+                        networkAddress.toResolvedInetSocketAddress(), pid, 5));
                 ji.reportPartitionAvailability(channel);
             }
         } catch (Exception e) {
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 f277046..6225d4c 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
@@ -18,8 +18,6 @@
  */
 package org.apache.hyracks.control.nc.work;
 
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.List;
@@ -301,12 +299,9 @@
                         NetworkAddress networkAddress = inputAddresses[i][j];
                         PartitionId pid = new PartitionId(jobId, inputs.get(i).getConnectorId(), j,
                                 td.getTaskAttemptId().getTaskId().getPartition());
-                        PartitionChannel channel = new PartitionChannel(pid,
-                                new NetworkInputChannel(ncs.getNetworkManager(),
-                                        new InetSocketAddress(
-                                                InetAddress.getByAddress(networkAddress.lookupIpAddress()),
-                                                networkAddress.getPort()),
-                                        pid, 5));
+                        PartitionChannel channel =
+                                new PartitionChannel(pid, new NetworkInputChannel(ncs.getNetworkManager(),
+                                        networkAddress.toResolvedInetSocketAddress(), pid, 5));
                         channels.add(channel);
                     }
                 }
@@ -318,6 +313,6 @@
 
     @Override
     public String toString() {
-        return getName() + " jobId:" + jobId;
+        return getName() + " jobId:" + jobId + " tasks:" + taskDescriptors.size();
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-dataflow-common/pom.xml
index 6bb86ef..4978e85 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/pom.xml
@@ -69,6 +69,7 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-control-nc</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>commons-io</groupId>
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml b/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
index 8ac7a12..fa668f0 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
@@ -74,6 +74,7 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-control-nc</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
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/base/AbstractReplicateOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractReplicateOperatorDescriptor.java
index 4c728ce..666c250 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractReplicateOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractReplicateOperatorDescriptor.java
@@ -35,10 +35,11 @@
 import org.apache.hyracks.dataflow.std.misc.MaterializerTaskState;
 
 /**
- * Abstract class for two replication related operator descriptor - replicate and split
+ * Abstract class for three replication related operator descriptors - replicate, split, and switch.
  * Replicate operator propagates all frames to all output branches.
  * That is, each tuple will be propagated to all output branches.
  * Split operator propagates each tuple in a frame to one output branch only.
+ * Switch is a generalization of split that propagates tuples based on a given output mapping.
  */
 public abstract class AbstractReplicateOperatorDescriptor extends AbstractOperatorDescriptor {
     protected static final long serialVersionUID = 1L;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractUnaryInputUnaryOutputIntrospectingOperatorNodePushable.java
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractUnaryInputUnaryOutputIntrospectingOperatorNodePushable.java
index 6afbccb..7f0f5a9 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractUnaryInputUnaryOutputIntrospectingOperatorNodePushable.java
@@ -16,17 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
+package org.apache.hyracks.dataflow.std.base;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+import org.apache.hyracks.api.dataflow.ISelfProfilingNodePushable;
+import org.apache.hyracks.api.dataflow.ITimedWriter;
 
-    void pause();
+public abstract class AbstractUnaryInputUnaryOutputIntrospectingOperatorNodePushable
+        extends AbstractUnaryInputUnaryOutputOperatorNodePushable implements ISelfProfilingNodePushable, ITimedWriter {
 
-    void resume();
 }
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 3645855..190baa2 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
@@ -117,9 +117,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 722512a..9cabb8a 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
@@ -290,17 +290,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..6e58e9c 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
@@ -30,14 +30,12 @@
 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;
 import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import org.apache.hyracks.dataflow.std.sort.AbstractExternalSortRunMerger;
 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;
 
 /**
  * This Operator pushes group-by aggregation into the external sort.
@@ -143,12 +141,11 @@
             @Override
             protected IRunGenerator getRunGenerator(IHyracksTaskContext ctx,
                     IRecordDescriptorProvider recordDescriptorProvider) throws HyracksDataException {
-                final boolean profile = ctx.getJobFlags().contains(JobFlag.PROFILE_RUNTIME);
                 IRunGenerator runGen = new ExternalSortGroupByRunGenerator(ctx, sortFields,
                         recordDescriptorProvider.getInputRecordDescriptor(this.getActivityId(), 0), framesLimit,
                         groupFields, keyNormalizerFactories, comparatorFactories, partialAggregatorFactory,
                         partialAggRecordDesc, ALG);
-                return profile ? TimedRunGenerator.time(runGen, ctx, "GroupBy (Sort Runs)") : runGen;
+                return runGen;
             }
         };
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/intersect/IntersectOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/intersect/IntersectOperatorDescriptor.java
index 67b0686..dc2b46f 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/intersect/IntersectOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/intersect/IntersectOperatorDescriptor.java
@@ -224,6 +224,11 @@
         }
 
         @Override
+        public String getDisplayName() {
+            return "Intersect";
+        }
+
+        @Override
         public IFrameWriter getInputFrameWriter(final int index) {
             return new IFrameWriter() {
                 private final int[] normalizedKey1 =
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 5f80165..04f5fe8 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;
@@ -94,6 +95,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,
@@ -190,7 +192,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);
     }
@@ -268,8 +273,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);
                 }
             }
@@ -424,6 +433,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)) {
@@ -527,7 +540,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.
@@ -550,6 +566,9 @@
 
             throw new HyracksDataException("The given tuple is too big");
         }
+        if (stats != null) {
+            stats.getBytesWritten().update(bigFrameAppender.getBuffer().limit());
+        }
         bigFrameAppender.write(runFileWriter, true);
     }
 
@@ -630,4 +649,8 @@
         LOGGER.debug("can't insert tuple in join memory. {}", details);
         LOGGER.debug("partitions status:\n{}", spillPolicy.partitionsStatus());
     }
+
+    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..38320ec 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
@@ -30,7 +30,6 @@
 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;
 import org.apache.hyracks.dataflow.std.buffermanager.EnumFreeSlotPolicy;
 
@@ -79,10 +78,9 @@
             @Override
             protected IRunGenerator getRunGenerator(IHyracksTaskContext ctx,
                     IRecordDescriptorProvider recordDescProvider) throws HyracksDataException {
-                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 runGen;
             }
         };
     }
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..934ae60 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
@@ -30,7 +30,6 @@
 import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
 
 public class TopKSorterOperatorDescriptor extends AbstractSorterOperatorDescriptor {
@@ -62,11 +61,9 @@
             @Override
             protected IRunGenerator getRunGenerator(IHyracksTaskContext ctx,
                     IRecordDescriptorProvider recordDescProvider) {
-                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;
-
+                return runGen;
             }
         };
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
index 90ea341..e4de6f2 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
@@ -56,41 +56,49 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-dataflow-std</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-control-cc</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-control-nc</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-btree</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-rtree</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-lsm-btree</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-lsm-rtree</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
@@ -102,11 +110,13 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-data-std</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-client</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>com.e-movimento.tinytools</groupId>
@@ -122,31 +132,37 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-dataflow-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-control-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-lsm-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-ipc</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>junit</groupId>
@@ -176,15 +192,18 @@
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-databind</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-util</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
+      <scope>test</scope>
     </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-shutdown-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-shutdown-test/pom.xml
index 7117fee..6c19207 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-shutdown-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-shutdown-test/pom.xml
@@ -43,6 +43,11 @@
             <usedDependency>org.apache.hyracks:hyracks-control-nc</usedDependency>
             <usedDependency>org.apache.hyracks:hyracks-control-cc</usedDependency>
           </usedDependencies>
+          <ignoredNonTestScopedDependencies>
+            <ignoredNonTestScopedDependency>org.apache.hyracks:hyracks-ipc:*</ignoredNonTestScopedDependency>
+            <ignoredNonTestScopedDependency>org.apache.hyracks:hyracks-api:*</ignoredNonTestScopedDependency>
+            <ignoredNonTestScopedDependency>org.apache.logging.log4j:log4j-api:*</ignoredNonTestScopedDependency>
+          </ignoredNonTestScopedDependencies>
         </configuration>
         <executions>
           <execution>
@@ -197,7 +202,6 @@
       <artifactId>hyracks-api</artifactId>
       <version>${project.version}</version>
       <type>jar</type>
-      <scope>compile</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
diff --git a/hyracks-fullstack/hyracks/hyracks-http/pom.xml b/hyracks-fullstack/hyracks/hyracks-http/pom.xml
index e0c8efe..20c30d0 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-http/pom.xml
@@ -82,7 +82,6 @@
     <dependency>
       <groupId>org.apache.httpcomponents</groupId>
       <artifactId>httpcore-nio</artifactId>
-      <version>4.4.15</version>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/api/IServletRequest.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/api/IServletRequest.java
index a049412..59c0ae0 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/api/IServletRequest.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/api/IServletRequest.java
@@ -23,6 +23,7 @@
 import java.util.Map;
 import java.util.Set;
 
+import io.netty.channel.Channel;
 import io.netty.handler.codec.http.FullHttpRequest;
 import io.netty.handler.codec.http.HttpScheme;
 
@@ -78,7 +79,9 @@
      * @param name header name
      * @return the header or null if not found
      */
-    String getHeader(CharSequence name);
+    default String getHeader(CharSequence name) {
+        return getHttpRequest().headers().get(name);
+    }
 
     /**
      * Get a request header if found, return the default value, otherwise
@@ -110,4 +113,6 @@
      * @return the remote address
      */
     InetSocketAddress getLocalAddress();
+
+    Channel getChannel();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/BaseRequest.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/BaseRequest.java
index c3c7dbc..4a72d53 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/BaseRequest.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/BaseRequest.java
@@ -29,6 +29,7 @@
 
 import org.apache.hyracks.http.api.IServletRequest;
 
+import io.netty.channel.Channel;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.handler.codec.http.FullHttpRequest;
 import io.netty.handler.codec.http.HttpScheme;
@@ -37,26 +38,22 @@
 public class BaseRequest implements IServletRequest {
 
     private static final List<String> NO_PARAM = Collections.singletonList(null);
+    private final Channel channel;
     protected final FullHttpRequest request;
     protected final Map<? extends CharSequence, List<String>> parameters;
-    protected final InetSocketAddress remoteAddress;
     protected final HttpScheme scheme;
-    protected final InetSocketAddress localAddress;
 
     public static IServletRequest create(ChannelHandlerContext ctx, FullHttpRequest request, HttpScheme scheme,
             boolean ignoreQueryParameters) {
         Map<? extends CharSequence, List<String>> param =
                 ignoreQueryParameters ? Collections.emptyMap() : new QueryStringDecoder(request.uri()).parameters();
-        InetSocketAddress remoteAddress = (InetSocketAddress) ctx.channel().remoteAddress();
-        InetSocketAddress localAddress = (InetSocketAddress) ctx.channel().localAddress();
-        return new BaseRequest(request, localAddress, remoteAddress, param, scheme);
+        return new BaseRequest(ctx.channel(), request, param, scheme);
     }
 
-    protected BaseRequest(FullHttpRequest request, InetSocketAddress localAddress, InetSocketAddress remoteAddress,
+    protected BaseRequest(Channel channel, FullHttpRequest request,
             Map<? extends CharSequence, List<String>> parameters, HttpScheme scheme) {
+        this.channel = channel;
         this.request = request;
-        this.localAddress = localAddress;
-        this.remoteAddress = remoteAddress;
         this.parameters = parameters;
         this.scheme = scheme;
     }
@@ -104,7 +101,7 @@
 
     @Override
     public InetSocketAddress getRemoteAddress() {
-        return remoteAddress;
+        return (InetSocketAddress) channel.remoteAddress();
     }
 
     @Override
@@ -114,6 +111,11 @@
 
     @Override
     public InetSocketAddress getLocalAddress() {
-        return localAddress;
+        return (InetSocketAddress) channel.localAddress();
+    }
+
+    @Override
+    public Channel getChannel() {
+        return channel;
     }
 }
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..a24ed95 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,12 +43,13 @@
 //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());
-    private StringBuilder logLineBuilder;
+    private final StringBuilder logLineBuilder;
 
+    private final Logger accessLogger;
     private String clientIp;
     private Instant requestTime;
     private String reqLine;
@@ -58,7 +59,12 @@
     private boolean lastChunk = false;
 
     public CLFLogger() {
+        this(LOGGER);
+    }
+
+    public CLFLogger(Logger accessLogger) {
         this.logLineBuilder = new StringBuilder();
+        this.accessLogger = accessLogger;
         respSize = 0;
     }
 
@@ -66,9 +72,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;
         }
@@ -109,21 +120,22 @@
     @Override
     public void flush(ChannelHandlerContext ctx) throws Exception {
         if (lastChunk) {
-            printAndPrepare();
+            printAndPrepare(ctx);
             lastChunk = false;
         }
         ctx.flush();
     }
 
-    private void printAndPrepare() {
+    private void printAndPrepare(ChannelHandlerContext ctx) {
         if (!accessLogger.isEnabled(ACCESS_LOG_LEVEL)) {
             return;
         }
         logLineBuilder.append(clientIp);
         //identd value - not relevant here
         logLineBuilder.append(" - ");
+        logLineBuilder.append(getUserId(ctx));
         //no http auth or any auth either for that matter
-        logLineBuilder.append(" - [");
+        logLineBuilder.append(" [");
         logLineBuilder.append(DATE_TIME_FORMATTER.format(requestTime));
         logLineBuilder.append("] \"");
         logLineBuilder.append(reqLine);
@@ -135,4 +147,8 @@
         respSize = 0;
         logLineBuilder.setLength(0);
     }
+
+    protected String getUserId(ChannelHandlerContext ctx) {
+        return "-";
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/FormUrlEncodedRequest.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/FormUrlEncodedRequest.java
index fa7ef66..983a1fd 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/FormUrlEncodedRequest.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/FormUrlEncodedRequest.java
@@ -18,7 +18,6 @@
  */
 package org.apache.hyracks.http.server;
 
-import java.net.InetSocketAddress;
 import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.LinkedHashMap;
@@ -29,6 +28,7 @@
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.server.utils.HttpUtil;
 
+import io.netty.channel.Channel;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.handler.codec.http.FullHttpRequest;
 import io.netty.handler.codec.http.HttpScheme;
@@ -46,13 +46,11 @@
             new QueryStringDecoder(request.uri()).parameters()
                     .forEach((name, value) -> parameters.computeIfAbsent(name, a -> new ArrayList<>()).addAll(value));
         }
-        InetSocketAddress remoteAddress = (InetSocketAddress) ctx.channel().remoteAddress();
-        InetSocketAddress localAddress = (InetSocketAddress) ctx.channel().localAddress();
-        return new FormUrlEncodedRequest(request, localAddress, remoteAddress, parameters, scheme);
+        return new FormUrlEncodedRequest(ctx.channel(), request, parameters, scheme);
     }
 
-    private FormUrlEncodedRequest(FullHttpRequest request, InetSocketAddress localAddress,
-            InetSocketAddress remoteAddress, Map<String, List<String>> parameters, HttpScheme scheme) {
-        super(request, localAddress, remoteAddress, parameters, scheme);
+    private FormUrlEncodedRequest(Channel channel, FullHttpRequest request, Map<String, List<String>> parameters,
+            HttpScheme scheme) {
+        super(channel, request, parameters, scheme);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerInitializer.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerInitializer.java
index b7b33a7..ad8a61f 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerInitializer.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerInitializer.java
@@ -49,8 +49,12 @@
         p.addLast(new HttpRequestCapacityController(server));
         p.addLast(new HttpRequestDecoder(maxRequestInitialLineLength, maxRequestHeaderSize, maxRequestChunkSize));
         p.addLast(new HttpResponseEncoder());
-        p.addLast(new CLFLogger());
+        p.addLast(getCLFLogger());
         p.addLast(new HttpRequestAggregator(maxRequestSize));
         p.addLast(server.createHttpHandler(maxResponseChunkSize));
     }
+
+    protected CLFLogger getCLFLogger() {
+        return new CLFLogger();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityManager.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityManager.java
index 42dacf5..db524ca 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityManager.java
@@ -36,7 +36,7 @@
 public class NetworkSecurityManager implements INetworkSecurityManager {
 
     private volatile INetworkSecurityConfig config;
-    private final ISocketChannelFactory sslSocketFactory;
+    protected final ISocketChannelFactory sslSocketFactory;
     public static final String TSL_VERSION = "TLSv1.2";
 
     public NetworkSecurityManager(INetworkSecurityConfig config) {
diff --git a/hyracks-fullstack/hyracks/hyracks-net/pom.xml b/hyracks-fullstack/hyracks/hyracks-net/pom.xml
index f1cdbf8..fbe7a5b 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-net/pom.xml
@@ -50,6 +50,7 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-ipc</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>junit</groupId>
diff --git a/hyracks-fullstack/hyracks/hyracks-server/pom.xml b/hyracks-fullstack/hyracks/hyracks-server/pom.xml
index baa8e3a..70db6a6 100644
--- a/hyracks-fullstack/hyracks/hyracks-server/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-server/pom.xml
@@ -44,21 +44,12 @@
     <plugins>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-jar-plugin</artifactId>
-        <executions>
-          <execution>
-            <goals>
-              <goal>test-jar</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-dependency-plugin</artifactId>
         <configuration>
           <usedDependencies combine.children="append">
             <usedDependency>org.apache.hyracks:hyracks-control-nc</usedDependency>
+            <usedDependency>org.apache.hyracks:hyracks-control-cc</usedDependency>
+            <usedDependency>org.apache.hyracks:hyracks-nc-service</usedDependency>
           </usedDependencies>
         </configuration>
         <executions>
@@ -71,23 +62,6 @@
         </executions>
       </plugin>
       <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-failsafe-plugin</artifactId>
-        <configuration>
-          <runOrder>alphabetical</runOrder>
-          <forkMode>pertest</forkMode>
-        </configuration>
-        <executions>
-          <execution>
-            <goals>
-              <goal>integration-test</goal>
-              <goal>verify</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-
-      <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
         <executions>
@@ -156,34 +130,8 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
-      <artifactId>hyracks-util</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.httpcomponents</groupId>
-      <artifactId>httpcore</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.httpcomponents</groupId>
-      <artifactId>httpclient</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-control-nc</artifactId>
       <version>${project.version}</version>
     </dependency>
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.logging.log4j</groupId>
-      <artifactId>log4j-api</artifactId>
-    </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
index 148694d..54ee271 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
@@ -103,6 +103,7 @@
     <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-lang3</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
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/IndexDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
index 16461de..b79c3b1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
@@ -73,8 +73,7 @@
         // Get local resource
         LocalResource lr = getResource();
         if (lr == null) {
-            LOGGER.error("index {} does not exist", resourceRef.getRelativePath());
-            throw HyracksDataException.create(ErrorCode.INDEX_DOES_NOT_EXIST);
+            throw HyracksDataException.create(ErrorCode.INDEX_DOES_NOT_EXIST, resourceRef.getRelativePath());
         }
         IResource resource = lr.getResource();
         index = resource.createInstance(ctx);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexLifecycleManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexLifecycleManager.java
index c9505a6..ab301a9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexLifecycleManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexLifecycleManager.java
@@ -196,7 +196,7 @@
     public void unregister(String resourcePath) throws HyracksDataException {
         IndexInfo info = indexInfos.get(resourcePath);
         if (info == null) {
-            throw HyracksDataException.create(ErrorCode.INDEX_DOES_NOT_EXIST);
+            throw HyracksDataException.create(ErrorCode.INDEX_DOES_NOT_EXIST, resourcePath);
         }
 
         if (info.referenceCount != 0) {
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..80cfe81
--- /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.2-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/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/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/AbstractLSMDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
index 4ee1245..966d6d1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
@@ -145,7 +145,7 @@
             // However, we cannot throw an exception here to be compatible with legacy datasets.
             // In this case, the disk component would always get a garbage Id [-1, -1], which makes the
             // component Id-based optimization useless but still correct.
-            LOGGER.warn("Component Id not found from disk component metadata");
+            LOGGER.warn("component id {} not found from disk component metadata {}", componentId, getIndex());
         }
         return componentId;
     }
@@ -160,9 +160,7 @@
     @Override
     public void markAsValid(boolean persist, IPageWriteFailureCallback callback) throws HyracksDataException {
         ComponentUtils.markAsValid(getMetadataHolder(), persist, callback);
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Marked as valid component with id: " + getId());
-        }
+        LOGGER.debug("marked {} as valid component with id {}", getIndex(), getId());
     }
 
     @Override
@@ -247,6 +245,7 @@
 
     @Override
     public String toString() {
-        return "{\"class\":" + getClass().getSimpleName() + "\", \"index\":" + getIndex().toString() + "}";
+        return "{\"class\":" + getClass().getSimpleName() + "\", \"id\":" + componentId + ", \"index\":" + getIndex()
+                + "}";
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
index bf93dc0..77e5c6e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
@@ -676,7 +676,8 @@
         return "{\"class\" : \"" + getClass().getSimpleName() + "\", \"dir\" : \"" + fileManager.getBaseDir()
                 + "\", \"memory\" : " + (memoryComponents == null ? 0 : memoryComponents) + ", \"disk\" : "
                 + diskComponents.size() + ", \"num-scheduled-flushes\":" + numScheduledFlushes
-                + ", \"current-memory-component\":" + currentMutableComponentId.get() + "}";
+                + ", \"current-memory-component\":"
+                + (currentMutableComponentId == null ? "" : currentMutableComponentId.get()) + "}";
     }
 
     @Override
@@ -858,11 +859,8 @@
         if (!memoryComponent.isModified() || opCtx.getOperation() == IndexOperation.DELETE_COMPONENTS) {
             return EmptyComponent.INSTANCE;
         }
-        if (LOGGER.isInfoEnabled()) {
-            FlushOperation flushOp = (FlushOperation) operation;
-            LOGGER.log(Level.INFO,
-                    "Flushing component with id: " + flushOp.getFlushingComponent().getId() + " in the index " + this);
-        }
+        LOGGER.debug("flushing component with id {} in the index {}",
+                ((FlushOperation) operation).getFlushingComponent().getId(), this);
         return doFlush(operation);
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
index 8d37d97..7088791 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
@@ -337,9 +337,7 @@
             throw new IllegalStateException(
                     this + " receives illegal id. Old id " + this.componentId + ", new id " + componentId);
         }
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Component Id was reset from " + this.componentId + " to " + componentId);
-        }
+        LOGGER.debug("component id of {} was reset from {} to {}", getIndex(), this.componentId, componentId);
         this.componentId = componentId;
         if (componentId != null) {
             LSMComponentIdUtils.persist(this.componentId, metadata);
@@ -355,6 +353,6 @@
     public String toString() {
         return "{\"class\":\"" + getClass().getSimpleName() + "\", \"state\":\"" + state + "\", \"writers\":"
                 + writerCount + ", \"readers\":" + readerCount + ", \"pendingFlushes\":" + pendingFlushes
-                + ", \"id\":\"" + componentId + "\"}";
+                + ", \"id\":\"" + componentId + "\", \"index\":" + getIndex() + "}";
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
index 3ea0f49..9a112ee 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
@@ -152,4 +152,9 @@
     public int getReaderCount() {
         return 0;
     }
+
+    @Override
+    public String toString() {
+        return "EmptyComponent";
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
index 950a8e5..717bcce 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
@@ -527,18 +527,19 @@
                 }
             }
         }
+        ILSMDiskComponent newComponent;
         try {
             doIo(operation);
         } finally {
-            exitComponents(operation.getAccessor().getOpContext(), LSMOperationType.FLUSH, operation.getNewComponent(),
+            newComponent = operation.getNewComponent();
+            exitComponents(operation.getAccessor().getOpContext(), LSMOperationType.FLUSH, newComponent,
                     operation.getStatus() == LSMIOOperationStatus.FAILURE);
             opTracker.completeOperation(lsmIndex, LSMOperationType.FLUSH,
                     operation.getAccessor().getOpContext().getSearchOperationCallback(),
                     operation.getAccessor().getOpContext().getModificationCallback());
         }
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Finished the flush operation for index: {}. Result: {}", lsmIndex, operation.getStatus());
-        }
+        LOGGER.debug("Finished the flush operation for {}. Result: {}",
+                (newComponent == null ? lsmIndex : newComponent), operation.getStatus());
     }
 
     public void doIo(ILSMIOOperation operation) {
@@ -577,24 +578,23 @@
 
     @Override
     public void merge(ILSMIOOperation operation) throws HyracksDataException {
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Started a merge operation for index: {}", lsmIndex);
-        }
+        LOGGER.debug("Started a merge operation for index {}", lsmIndex);
         synchronized (opTracker) {
             enterComponents(operation.getAccessor().getOpContext(), LSMOperationType.MERGE);
         }
+        ILSMDiskComponent newComponent;
         try {
             doIo(operation);
         } finally {
-            exitComponents(operation.getAccessor().getOpContext(), LSMOperationType.MERGE, operation.getNewComponent(),
+            newComponent = operation.getNewComponent();
+            exitComponents(operation.getAccessor().getOpContext(), LSMOperationType.MERGE, newComponent,
                     operation.getStatus() == LSMIOOperationStatus.FAILURE);
             opTracker.completeOperation(lsmIndex, LSMOperationType.MERGE,
                     operation.getAccessor().getOpContext().getSearchOperationCallback(),
                     operation.getAccessor().getOpContext().getModificationCallback());
         }
-        if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Finished the merge operation for index: {}. Result: {}", lsmIndex, operation.getStatus());
-        }
+        LOGGER.debug("Finished the merge operation for {}. Result: {}",
+                (newComponent == null ? lsmIndex : newComponent), operation.getStatus());
     }
 
     @Override
@@ -777,6 +777,8 @@
                 ioOperation = scheduleFlush(ctx);
             } else {
                 // since we're not deleting the memory component, we can't delete any previous component
+                LOGGER.debug("not deleting any components of {} since memory component {} won't be deleted", lsmIndex,
+                        memComponent);
                 return;
             }
         }
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-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
index e77acea..959bb50 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
@@ -70,6 +70,11 @@
     }
 
     @Override
+    public ICachedPage pin(long dpid, boolean newPage, boolean incrementStats) throws HyracksDataException {
+        return vbc.pin(dpid, newPage);
+    }
+
+    @Override
     public void unpin(ICachedPage page) throws HyracksDataException {
         vbc.unpin(page);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
index 5871b31..e222461 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
@@ -260,6 +260,11 @@
         return page;
     }
 
+    @Override
+    public ICachedPage pin(long dpid, boolean newPage, boolean incrementStats) throws HyracksDataException {
+        return pin(dpid, newPage);
+    }
+
     private int hash(long dpid) {
         int hashValue = (int) dpid ^ (Integer.reverse((int) (dpid >>> 32)) >>> 1);
         return hashValue % buckets.length;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/ComponentUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/ComponentUtils.java
index 1ff9fa8..842ec61 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/ComponentUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/ComponentUtils.java
@@ -34,7 +34,6 @@
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.IPageWriteFailureCallback;
-import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -80,11 +79,7 @@
      */
     public static void get(ILSMIndex index, IValueReference key, ArrayBackedValueStorage value)
             throws HyracksDataException {
-        boolean loggable = LOGGER.isDebugEnabled();
         value.reset();
-        if (loggable) {
-            LOGGER.log(Level.DEBUG, "Getting " + key + " from index " + index);
-        }
         // Lock the opTracker to ensure index components don't change
         synchronized (index.getOperationTracker()) {
             ILSMMemoryComponent cmc = index.getCurrentMemoryComponent();
@@ -92,33 +87,17 @@
                 index.getCurrentMemoryComponent().getMetadata().get(key, value);
             }
             if (value.getLength() == 0) {
-                if (loggable) {
-                    LOGGER.log(Level.DEBUG, key + " was not found in mutable memory component of " + index);
-                }
-                // was not found in the in current mutable component, search in the other in memory components
+                // was not found in the in current mutable component, search in the other in-memory components
                 fromImmutableMemoryComponents(index, key, value);
                 if (value.getLength() == 0) {
-                    if (loggable) {
-                        LOGGER.log(Level.DEBUG, key + " was not found in all immmutable memory components of " + index);
-                    }
-                    // was not found in the in all in memory components, search in the disk components
+                    // was not found in all in-memory components, search in the disk components
                     fromDiskComponents(index, key, value);
-                    if (loggable) {
-                        if (value.getLength() == 0) {
-                            LOGGER.log(Level.DEBUG, key + " was not found in all disk components of " + index);
-                        } else {
-                            LOGGER.log(Level.DEBUG, key + " was found in disk components of " + index);
-                        }
-                    }
-                } else {
-                    if (loggable) {
-                        LOGGER.log(Level.DEBUG, key + " was found in the immutable memory components of " + index);
+                    if (value.getLength() == 0) {
+                        LOGGER.debug("{} was NOT found", key);
                     }
                 }
             } else {
-                if (loggable) {
-                    LOGGER.log(Level.DEBUG, key + " was found in mutable memory component of " + index);
-                }
+                LOGGER.debug("{} was found in mutable memory component {}", key, cmc);
             }
         }
     }
@@ -143,17 +122,11 @@
 
     private static void fromDiskComponents(ILSMIndex index, IValueReference key, ArrayBackedValueStorage value)
             throws HyracksDataException {
-        boolean loggable = LOGGER.isDebugEnabled();
-        if (loggable) {
-            LOGGER.log(Level.DEBUG, "Getting " + key + " from disk components of " + index);
-        }
         for (ILSMDiskComponent c : index.getDiskComponents()) {
-            if (loggable) {
-                LOGGER.log(Level.DEBUG, "Getting " + key + " from disk components " + c);
-            }
             c.getMetadata().get(key, value);
             if (value.getLength() != 0) {
                 // Found
+                LOGGER.debug("{} was found in disk component {}", key, c);
                 return;
             }
         }
@@ -161,21 +134,10 @@
 
     private static void fromImmutableMemoryComponents(ILSMIndex index, IValueReference key,
             ArrayBackedValueStorage value) throws HyracksDataException {
-        boolean loggable = LOGGER.isDebugEnabled();
-        if (loggable) {
-            LOGGER.log(Level.DEBUG, "Getting " + key + " from immutable memory components of " + index);
-        }
         List<ILSMMemoryComponent> memComponents = index.getMemoryComponents();
         int numOtherMemComponents = memComponents.size() - 1;
         int next = index.getCurrentMemoryComponentIndex();
-        if (loggable) {
-            LOGGER.log(Level.DEBUG, index + " has " + numOtherMemComponents + " immutable memory components");
-        }
         for (int i = 0; i < numOtherMemComponents; i++) {
-            if (loggable) {
-                LOGGER.log(Level.DEBUG,
-                        "trying to get " + key + " from immutable memory components number: " + (i + 1));
-            }
             next = next - 1;
             if (next < 0) {
                 next = memComponents.size() - 1;
@@ -185,6 +147,7 @@
                 c.getMetadata().get(key, value);
                 if (value.getLength() != 0) {
                     // Found
+                    LOGGER.debug("{} was found in immutable memory component {}", key, c);
                     return;
                 }
             }
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 ea108cb..e0999d4 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
@@ -166,13 +166,18 @@
 
     @Override
     public ICachedPage pin(long dpid, boolean newPage) throws HyracksDataException {
+        return pin(dpid, newPage, true);
+    }
+
+    @Override
+    public ICachedPage pin(long dpid, boolean newPage, boolean incrementStats) throws HyracksDataException {
         // Calling the pinSanityCheck should be used only for debugging, since
         // the synchronized block over the fileInfoMap is a hot spot.
         if (DEBUG) {
             pinSanityCheck(dpid);
         }
         final IThreadStats threadStats = statsSubscribers.get(Thread.currentThread());
-        if (threadStats != null) {
+        if (threadStats != null && incrementStats) {
             threadStats.pagePinned();
         }
         CachedPage cPage = findPage(dpid);
@@ -194,7 +199,7 @@
             synchronized (cPage) {
                 if (!cPage.valid) {
                     try {
-                        tryRead(cPage);
+                        tryRead(cPage, incrementStats);
                         cPage.valid = true;
                     } catch (Exception e) {
                         LOGGER.log(ExceptionUtils.causedByInterrupt(e) ? Level.DEBUG : Level.WARN,
@@ -521,10 +526,10 @@
         return false;
     }
 
-    private void tryRead(CachedPage cPage) throws HyracksDataException {
+    private void tryRead(CachedPage cPage, boolean incrementStats) throws HyracksDataException {
         for (int i = 1; i <= MAX_PAGE_READ_ATTEMPTS; i++) {
             try {
-                read(cPage);
+                read(cPage, incrementStats);
                 return;
             } catch (HyracksDataException readException) {
                 if (readException.matches(ErrorCode.CANNOT_READ_CLOSED_FILE) && i != MAX_PAGE_READ_ATTEMPTS) {
@@ -548,10 +553,14 @@
         }
     }
 
-    private void read(CachedPage cPage) throws HyracksDataException {
+    private void read(CachedPage cPage, boolean incrementStats) throws HyracksDataException {
         BufferedFileHandle fInfo = getFileHandle(cPage);
         cPage.buffer.clear();
         fInfo.read(cPage);
+        final IThreadStats threadStats = statsSubscribers.get(Thread.currentThread());
+        if (threadStats != null && incrementStats) {
+            threadStats.coldRead();
+        }
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
index 8c3d492..c76c781 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
@@ -77,13 +77,18 @@
     }
 
     @Override
-    public ICachedPage pin(long dpid, boolean newPage) throws HyracksDataException {
-        ICachedPage page = bufferCache.pin(dpid, newPage);
+    public ICachedPage pin(long dpid, boolean newPage, boolean incrementStats) throws HyracksDataException {
+        ICachedPage page = bufferCache.pin(dpid, newPage, incrementStats);
         pinCount.addAndGet(1);
         return page;
     }
 
     @Override
+    public ICachedPage pin(long dpid, boolean newPage) throws HyracksDataException {
+        return pin(dpid, newPage, true);
+    }
+
+    @Override
     public void unpin(ICachedPage page) throws HyracksDataException {
         bufferCache.unpin(page);
         unpinCount.addAndGet(1);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
index df0bea8..3e977bd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
@@ -105,6 +105,28 @@
     ICachedPage pin(long dpid, boolean newPage) throws HyracksDataException;
 
     /**
+     * Pin the page so it can't be evicted from the buffer cache...
+     *
+     * @param dpid
+     *            page id is a unique id that is a combination of file id and page id
+     * @param newPage
+     *            whether this page is expected to be new.
+     *            NOTE: undefined:
+     *            -- what if the flag is true but the page exists?
+     *            -- what if the flag is false but the page doesn't exist
+     *
+     * @param incrementStats
+     *            whether to increment the coldRead and pinCount counters when
+     *            the page is pinned. this is to not bias the count when using
+     *            accessors that cause nested pins due to wrapping file handles,
+     *            like compression
+     *
+     * @return the pinned page
+     * @throws HyracksDataException
+     */
+    ICachedPage pin(long dpid, boolean newPage, boolean incrementStats) throws HyracksDataException;
+
+    /**
      * Unpin a pinned page so its buffer can be recycled
      *
      * @param page
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/CompressedFileManager.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/CompressedFileManager.java
index 7d0cc62..b334b1f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/CompressedFileManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/compression/file/CompressedFileManager.java
@@ -311,7 +311,9 @@
         final int numOfEntriesPerPage = bufferCache.getPageSize() / ENTRY_LENGTH;
         //get the last page which may contain less entries than maxNumOfEntries
         final long dpid = getDiskPageId(numOfPages - 1);
-        final ICachedPage page = bufferCache.pin(dpid, false);
+        //exclude the LAF from the buffer cache pin/read stats, because it is hot in any
+        //sane scenario. otherwise it inflates the query's cache ratio greatly. 
+        final ICachedPage page = bufferCache.pin(dpid, false, false);
         try {
             final ByteBuffer buf = page.getBuffer();
 
@@ -330,7 +332,10 @@
 
     private ICachedPage pinAndGetPage(int compressedPageId) throws HyracksDataException {
         final int pageId = compressedPageId * ENTRY_LENGTH / bufferCache.getPageSize();
-        return bufferCache.pin(getDiskPageId(pageId), false);
+        //don't increment the stats with this pin. this is because we are pinning on behalf
+        //of a caller, so our pins will be an interfering access pattern that conflates itself
+        //with the caller's cache pattern.
+        return bufferCache.pin(getDiskPageId(pageId), false, false);
     }
 
     private long getDiskPageId(int pageId) {
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
similarity index 64%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
index 6afbccb..8ca1a82 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
@@ -16,17 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
+package org.apache.hyracks.storage.common.projection;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+import java.io.DataOutput;
+import java.io.IOException;
 
-    void pause();
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 
-    void resume();
+public interface ITupleProjector {
+    void project(ITupleReference tuple, DataOutput dos, ArrayTupleBuilder tb) throws IOException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
similarity index 61%
copy from hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
copy to hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
index 6afbccb..ff9ecf9 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IPassableTimer.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
@@ -16,17 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.api.dataflow;
+package org.apache.hyracks.storage.common.projection;
 
-public interface IPassableTimer {
-    /*
-    A timer intended to be used for timing the individual components of a
-    pipelined process. An instance of IPassableTimer is held by each method
-    in the pipeline, and is paused() when that method passes off control to
-    a component above it, and is resume()d when the component above it returns.
-     */
+import java.io.Serializable;
 
-    void pause();
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-    void resume();
+/**
+ * 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/LicensingTestBase.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/LicensingTestBase.java
index 283c12c..6588f5f 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/LicensingTestBase.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/LicensingTestBase.java
@@ -22,6 +22,7 @@
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.commons.io.FileUtils;
@@ -46,7 +47,7 @@
             }
             installerDir = FileUtil.joinPath(targetDir, list[0], topLevel[0]);
             Assert.assertNotNull("installerDir", list);
-            Assert.assertFalse("Ambiguous install dir (" + pattern + "): " + list, list.length > 1);
+            Assert.assertFalse("Ambiguous install dir (" + pattern + "): " + Arrays.toString(list), list.length > 1);
             Assert.assertEquals("Can't find install dir (" + pattern + ")", 1, topLevel.length);
         }
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestNCServiceContext.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestNCServiceContext.java
index 63126ab..7b0abc2 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestNCServiceContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestNCServiceContext.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.test.support;
 
 import java.io.Serializable;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ThreadFactory;
 
 import org.apache.hyracks.api.application.INCServiceContext;
@@ -26,8 +27,9 @@
 import org.apache.hyracks.api.application.IStateDumpHandler;
 import org.apache.hyracks.api.comm.IChannelInterfaceFactory;
 import org.apache.hyracks.api.config.IApplicationConfig;
+import org.apache.hyracks.api.deployment.DeploymentId;
 import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.api.job.IJobSerializerDeserializerContainer;
+import org.apache.hyracks.api.job.IJobSerializerDeserializer;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponentManager;
 import org.apache.hyracks.api.lifecycle.LifeCycleComponentManager;
 import org.apache.hyracks.api.messages.IMessageBroker;
@@ -102,7 +104,7 @@
     }
 
     @Override
-    public IJobSerializerDeserializerContainer getJobSerializerDeserializerContainer() {
+    public ConcurrentMap<DeploymentId, IJobSerializerDeserializer> getJobSerializerDeserializerContainer() {
         return null;
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-dataflow-common-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-dataflow-common-test/pom.xml
index 1fd0aed..467423a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-dataflow-common-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-dataflow-common-test/pom.xml
@@ -48,30 +48,36 @@
         <dependency>
             <groupId>com.fasterxml.jackson.core</groupId>
             <artifactId>jackson-databind</artifactId>
+            <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.commons</groupId>
             <artifactId>commons-lang3</artifactId>
+            <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.hyracks</groupId>
             <artifactId>hyracks-api</artifactId>
             <version>${project.version}</version>
+            <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.hyracks</groupId>
             <artifactId>hyracks-data-std</artifactId>
             <version>${project.version}</version>
+            <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.hyracks</groupId>
             <artifactId>hyracks-dataflow-common</artifactId>
             <version>${project.version}</version>
+            <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.hyracks</groupId>
             <artifactId>hyracks-test-support</artifactId>
             <version>${project.version}</version>
+            <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>junit</groupId>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/pom.xml
new file mode 100644
index 0000000..bd7bf75
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/pom.xml
@@ -0,0 +1,150 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+
+<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-server-test</artifactId>
+  <name>hyracks-server-test</name>
+  <parent>
+    <groupId>org.apache.hyracks</groupId>
+    <artifactId>hyracks-tests</artifactId>
+    <version>0.3.8.2-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>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <configuration>
+          <usedDependencies combine.children="append">
+            <usedDependency>org.apache.hyracks:hyracks-server</usedDependency>
+          </usedDependencies>
+        </configuration>
+        <executions>
+          <execution>
+            <phase>process-test-classes</phase>
+            <goals>
+              <goal>analyze-only</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-failsafe-plugin</artifactId>
+        <configuration>
+          <runOrder>alphabetical</runOrder>
+          <forkMode>pertest</forkMode>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>integration-test</goal>
+              <goal>verify</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-control-cc</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-nc-service</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-server</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+<!--
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-control-nc</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+-->
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpcore</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-util</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-api</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpclient</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>
diff --git a/hyracks-fullstack/hyracks/hyracks-server/src/test/java/org/apache/hyracks/test/server/NCServiceIT.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/java/org/apache/hyracks/test/server/NCServiceIT.java
similarity index 97%
rename from hyracks-fullstack/hyracks/hyracks-server/src/test/java/org/apache/hyracks/test/server/NCServiceIT.java
rename to hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/java/org/apache/hyracks/test/server/NCServiceIT.java
index cd0359f..5ae57cd 100644
--- a/hyracks-fullstack/hyracks/hyracks-server/src/test/java/org/apache/hyracks/test/server/NCServiceIT.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/java/org/apache/hyracks/test/server/NCServiceIT.java
@@ -47,7 +47,7 @@
     private static final String TARGET_DIR = FileUtil.joinPath(".", "target");
     private static final String LOG_DIR = FileUtil.joinPath(TARGET_DIR, "failsafe-reports");
     private static final String RESOURCE_DIR = FileUtil.joinPath(TARGET_DIR, "test-classes", "NCServiceIT");
-    private static final String APP_HOME = FileUtil.joinPath(TARGET_DIR, "appassembler");
+    private static final String APP_HOME = FileUtil.joinPath("..", "..", "hyracks-server", "target", "appassembler");
     private static final Logger LOGGER = LogManager.getLogger();
 
     private static HyracksVirtualCluster cluster = null;
diff --git a/hyracks-fullstack/hyracks/hyracks-server/src/test/java/org/apache/hyracks/test/server/process/HyracksCCProcess.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/java/org/apache/hyracks/test/server/process/HyracksCCProcess.java
similarity index 100%
rename from hyracks-fullstack/hyracks/hyracks-server/src/test/java/org/apache/hyracks/test/server/process/HyracksCCProcess.java
rename to hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/java/org/apache/hyracks/test/server/process/HyracksCCProcess.java
diff --git a/hyracks-fullstack/hyracks/hyracks-server/src/test/java/org/apache/hyracks/test/server/process/HyracksNCServiceProcess.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/java/org/apache/hyracks/test/server/process/HyracksNCServiceProcess.java
similarity index 100%
rename from hyracks-fullstack/hyracks/hyracks-server/src/test/java/org/apache/hyracks/test/server/process/HyracksNCServiceProcess.java
rename to hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/java/org/apache/hyracks/test/server/process/HyracksNCServiceProcess.java
diff --git a/hyracks-fullstack/hyracks/hyracks-server/src/test/java/org/apache/hyracks/test/server/process/HyracksServerProcess.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/java/org/apache/hyracks/test/server/process/HyracksServerProcess.java
similarity index 100%
rename from hyracks-fullstack/hyracks/hyracks-server/src/test/java/org/apache/hyracks/test/server/process/HyracksServerProcess.java
rename to hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/java/org/apache/hyracks/test/server/process/HyracksServerProcess.java
diff --git a/hyracks-fullstack/hyracks/hyracks-server/src/test/java/org/apache/hyracks/test/server/process/HyracksVirtualCluster.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/java/org/apache/hyracks/test/server/process/HyracksVirtualCluster.java
similarity index 100%
rename from hyracks-fullstack/hyracks/hyracks-server/src/test/java/org/apache/hyracks/test/server/process/HyracksVirtualCluster.java
rename to hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/java/org/apache/hyracks/test/server/process/HyracksVirtualCluster.java
diff --git a/hyracks-fullstack/hyracks/hyracks-server/src/test/resources/NCServiceIT/cc.conf b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/resources/NCServiceIT/cc.conf
similarity index 100%
rename from hyracks-fullstack/hyracks/hyracks-server/src/test/resources/NCServiceIT/cc.conf
rename to hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/resources/NCServiceIT/cc.conf
diff --git a/hyracks-fullstack/hyracks/hyracks-server/src/test/resources/NCServiceIT/nc-blue.conf b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/resources/NCServiceIT/nc-blue.conf
similarity index 100%
rename from hyracks-fullstack/hyracks/hyracks-server/src/test/resources/NCServiceIT/nc-blue.conf
rename to hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/resources/NCServiceIT/nc-blue.conf
diff --git a/hyracks-fullstack/hyracks/hyracks-server/src/test/resources/NCServiceIT/nc-red.conf b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/resources/NCServiceIT/nc-red.conf
similarity index 100%
rename from hyracks-fullstack/hyracks/hyracks-server/src/test/resources/NCServiceIT/nc-red.conf
rename to hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/resources/NCServiceIT/nc-red.conf
diff --git a/hyracks-fullstack/hyracks/hyracks-server/src/test/resources/logging.properties b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/resources/logging.properties
similarity index 100%
rename from hyracks-fullstack/hyracks/hyracks-server/src/test/resources/logging.properties
rename to hyracks-fullstack/hyracks/hyracks-tests/hyracks-server-test/src/test/resources/logging.properties
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/pom.xml
index 800fcdd..56658f6 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/pom.xml
@@ -48,6 +48,7 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-bloomfilter</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
@@ -59,16 +60,19 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-dataflow-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>junit</groupId>
@@ -78,6 +82,7 @@
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
+      <scope>test</scope>
     </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
index 038e346..fbb0589 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
@@ -49,21 +49,25 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-btree</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-test-support</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-dataflow-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
@@ -76,11 +80,13 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>junit</groupId>
@@ -91,10 +97,17 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-data-std</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+      <scope>test</scope>
     </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
index f191a2b..3794e3d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
@@ -69,11 +69,13 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-lsm-btree</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
@@ -86,67 +88,81 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-test-support</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-dataflow-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-lsm-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-control-nc</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-data-std</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-btree</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-bloomfilter</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-util</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-lang3</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-databind</artifactId>
+      <scope>test</scope>
     </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
index 8cad497..e697b54 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
@@ -101,6 +101,11 @@
     }
 
     @Override
+    public ICachedPage pin(long dpid, boolean newPage, boolean incrementStats) throws HyracksDataException {
+        return pin(dpid, newPage);
+    }
+
+    @Override
     public void unpin(ICachedPage page) throws HyracksDataException {
         vbc.unpin(page);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
index cbae843..f949dee 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
@@ -48,45 +48,54 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-lsm-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-test-support</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-control-nc</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-dataflow-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-data-std</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml
index 4be352f..3d6bd72 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml
@@ -48,6 +48,7 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
@@ -65,31 +66,37 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-util</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-dataflow-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-dataflow-std</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-lsm-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
@@ -101,6 +108,7 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>junit</groupId>
@@ -111,14 +119,17 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-btree</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
+      <scope>test</scope>
     </dependency>
   </dependencies>
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml
index 4db6a64..9e723ca 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml
@@ -48,59 +48,76 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-lsm-rtree</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-test-support</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-dataflow-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-lsm-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-rtree</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-control-nc</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-data-std</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+      <scope>test</scope>
     </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
index 718cb1b..6dd7aec 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
@@ -49,6 +49,7 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-rtree</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
@@ -60,11 +61,13 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-dataflow-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-am-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
@@ -77,11 +80,13 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>junit</groupId>
@@ -92,10 +97,23 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-data-std</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-control-nc</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+      <scope>test</scope>
     </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
index 86561c6..e84dda0 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
@@ -44,37 +44,45 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-storage-common</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-test-support</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-control-nc</artifactId>
       <version>${project.version}</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>commons-io</groupId>
       <artifactId>commons-io</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-lang3</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
+      <scope>test</scope>
     </dependency>
   </dependencies>
 </project>
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/pom.xml
index a26adea..71bb0e9 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/pom.xml
@@ -52,5 +52,6 @@
     <module>hyracks-storage-am-lsm-invertedindex-test</module>
     <module>hyracks-storage-am-bloomfilter-test</module>
     <module>hyracks-dataflow-common-test</module>
+    <module>hyracks-server-test</module>
   </modules>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
index 8f8e8f6..7eb9fd4 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
@@ -46,7 +46,7 @@
     public static final int EC_FAILED_TO_ABORT_METADATA_TXN = 7;
     public static final int EC_INCONSISTENT_METADATA = 8;
     public static final int EC_UNCAUGHT_THROWABLE = 9;
-    public static final int EC_UNHANDLED_EXCEPTION = 11;
+    public static final int EC_HALT_UNHANDLED_EXCEPTION = 11;
     public static final int EC_FAILED_TO_DELETE_CORRUPTED_RESOURCES = 12;
     public static final int EC_ERROR_CREATING_RESOURCES = 13;
     public static final int EC_TXN_LOG_FLUSHER_FAILURE = 14;
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/JSONUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/JSONUtil.java
index f937e97..191df1f 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/JSONUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/JSONUtil.java
@@ -50,6 +50,10 @@
         SORTED_MAPPER.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true);
     }
 
+    public static JsonNode readTree(byte[] bytes, int start, int size) throws IOException {
+        return OBJECT_MAPPER.readTree(bytes, start, size);
+    }
+
     public static String convertNode(final JsonNode node) throws JsonProcessingException {
         return SORTED_MAPPER.writeValueAsString(SORTED_MAPPER.treeToValue(node, Object.class));
     }
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/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/pom.xml b/hyracks-fullstack/hyracks/pom.xml
index a50e544..5d88a9a 100644
--- a/hyracks-fullstack/hyracks/pom.xml
+++ b/hyracks-fullstack/hyracks/pom.xml
@@ -35,7 +35,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-plugin-plugin</artifactId>
-          <version>3.6.0</version>
+          <version>3.9.0</version>
         </plugin>
         <!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
         <plugin>
@@ -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>
diff --git a/hyracks-fullstack/pom.xml b/hyracks-fullstack/pom.xml
index 487cadd..3453ebd 100644
--- a/hyracks-fullstack/pom.xml
+++ b/hyracks-fullstack/pom.xml
@@ -49,8 +49,7 @@
     <root.dir>${basedir}</root.dir>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <file.encoding>UTF-8</file.encoding>
-    <source.jdk.version>11</source.jdk.version>
-    <target.jdk.version>11</target.jdk.version>
+    <maven.compiler.release>11</maven.compiler.release>
     <javac.xlint.value>all</javac.xlint.value>
     <jvm.extraargs />
     <sonar.jacoco.reportPath>${env.PWD}/target/jacoco-merged.exec</sonar.jacoco.reportPath>
@@ -72,7 +71,7 @@
     <!-- Versions under dependencymanagement or used in many projects via properties -->
     <hadoop.version>3.3.6</hadoop.version>
     <jacoco.version>0.7.6.201602180812</jacoco.version>
-    <log4j.version>2.19.0</log4j.version>
+    <log4j.version>2.22.1</log4j.version>
     <snappy.version>1.1.10.5</snappy.version>
     <jackson.version>2.14.3</jackson.version>
     <jackson-databind.version>${jackson.version}</jackson-databind.version>
@@ -273,12 +272,12 @@
       <dependency>
         <groupId>commons-io</groupId>
         <artifactId>commons-io</artifactId>
-        <version>2.11.0</version>
+        <version>2.15.1</version>
       </dependency>
       <dependency>
         <groupId>org.apache.commons</groupId>
         <artifactId>commons-text</artifactId>
-        <version>1.10.0</version>
+        <version>1.11.0</version>
       </dependency>
       <dependency>
         <groupId>com.fasterxml.jackson.core</groupId>
@@ -325,7 +324,7 @@
       <dependency>
         <groupId>org.apache.commons</groupId>
         <artifactId>commons-lang3</artifactId>
-        <version>3.12.0</version>
+        <version>3.14.0</version>
       </dependency>
       <dependency>
         <groupId>org.apache.commons</groupId>
@@ -335,12 +334,17 @@
       <dependency>
         <groupId>org.apache.httpcomponents</groupId>
         <artifactId>httpcore</artifactId>
-        <version>4.4.15</version>
+        <version>4.4.16</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.httpcomponents</groupId>
+        <artifactId>httpcore-nio</artifactId>
+        <version>4.4.16</version>
       </dependency>
       <dependency>
         <groupId>org.apache.httpcomponents</groupId>
         <artifactId>httpclient</artifactId>
-        <version>4.5.13</version>
+        <version>4.5.14</version>
       </dependency>
       <dependency>
         <groupId>org.apache.rat</groupId>
@@ -425,7 +429,7 @@
       <dependency>
         <groupId>org.slf4j</groupId>
         <artifactId>slf4j-simple</artifactId>
-        <version>1.7.33</version>
+        <version>1.7.36</version>
       </dependency>
       <dependency>
         <groupId>org.apache.logging.log4j</groupId>
@@ -566,8 +570,6 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-compiler-plugin</artifactId>
         <configuration>
-          <source>${source.jdk.version}</source>
-          <target>${target.jdk.version}</target>
           <compilerArgument>-Xlint:${javac.xlint.value}</compilerArgument>
         </configuration>
       </plugin>
@@ -593,7 +595,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
-        <version>3.0.0</version>
+        <version>3.3.1</version>
         <executions>
           <execution>
             <id>verify-style</id>
@@ -677,7 +679,7 @@
                   <version>[3.3.9,)</version>
                 </requireMavenVersion>
                 <requireJavaVersion>
-                  <version>[11,)</version>
+                  <version>[${maven.compiler.release},)</version>
                 </requireJavaVersion>
               </rules>
             </configuration>
@@ -893,7 +895,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-dependency-plugin</artifactId>
-          <version>3.1.2</version>
+          <version>3.6.0</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.doxia</groupId>