Merge branch 'gerrit/neo'

Change-Id: I1944b0ce55e9aeaae4ec5c3c2947201a73902752
diff --git a/asterixdb/asterix-algebra/pom.xml b/asterixdb/asterix-algebra/pom.xml
index 7a32f42..f542af3 100644
--- a/asterixdb/asterix-algebra/pom.xml
+++ b/asterixdb/asterix-algebra/pom.xml
@@ -202,6 +202,10 @@
       <artifactId>hyracks-api</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-client</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
index 06a6687..fed180b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
@@ -62,9 +62,12 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeBatchPointSearchCursor;
 
 /**
  * Contributes the runtime operator for an unnest-map representing a BTree search.
@@ -157,7 +160,9 @@
                 nonMatchWriterFactory, dataset, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes,
                 jobGenParams.isLowKeyInclusive(), jobGenParams.isHighKeyInclusive(), propagateFilter,
                 nonFilterWriterFactory, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit,
-                unnestMap.getGenerateCallBackProceedResultVar(), isPrimaryIndexPointSearch(op));
+                unnestMap.getGenerateCallBackProceedResultVar(),
+                isPrimaryIndexPointSearch(op, context.getPhysicalOptimizationConfig()),
+                DefaultTupleProjectorFactory.INSTANCE);
         IOperatorDescriptor opDesc = btreeSearch.first;
         opDesc.setSourceLocation(unnestMap.getSourceLocation());
 
@@ -168,8 +173,12 @@
         builder.contributeGraphEdge(srcExchange, 0, unnestMap, 0);
     }
 
-    private boolean isPrimaryIndexPointSearch(ILogicalOperator op) {
-        if (!isEqCondition || !isPrimaryIndex || !lowKeyVarList.equals(highKeyVarList)) {
+    /**
+     * Check whether we can use {@link LSMBTreeBatchPointSearchCursor} to perform point-lookups on the primary index
+     */
+    private boolean isPrimaryIndexPointSearch(ILogicalOperator op, PhysicalOptimizationConfig config) {
+        if (!config.isBatchLookupEnabled() || !isEqCondition || !isPrimaryIndex
+                || !lowKeyVarList.equals(highKeyVarList)) {
             return false;
         }
         Index searchIndex = ((DataSourceIndex) idx).getIndex();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/DefaultRuleSetFactory.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/DefaultRuleSetFactory.java
index ba42036..76404ff 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/DefaultRuleSetFactory.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/DefaultRuleSetFactory.java
@@ -23,23 +23,33 @@
 
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.optimizer.base.RuleCollections;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialFirstRuleCheckFixpointRuleController;
 import org.apache.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialFixpointRuleController;
 import org.apache.hyracks.algebricks.compiler.rewriter.rulecontrollers.SequentialOnceRuleController;
 import org.apache.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
 
 public class DefaultRuleSetFactory implements IRuleSetFactory {
 
     @Override
     public List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getLogicalRewrites(
-            ICcApplicationContext appCtx) throws AlgebricksException {
+            ICcApplicationContext appCtx) {
         return buildLogical(appCtx);
     }
 
     @Override
+    public List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getLogicalRewrites(IRuleSetKind ruleSetKind,
+            ICcApplicationContext appCtx) {
+        if (ruleSetKind == RuleSetKind.SAMPLING) {
+            return buildLogicalSampling();
+        } else {
+            throw new IllegalArgumentException(String.valueOf(ruleSetKind));
+        }
+    }
+
+    @Override
     public List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getPhysicalRewrites(
             ICcApplicationContext appCtx) {
         return buildPhysical(appCtx);
@@ -76,6 +86,14 @@
         return defaultLogicalRewrites;
     }
 
+    public static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> buildLogicalSampling() {
+        List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites = new ArrayList<>();
+        SequentialFixpointRuleController seqCtrlNoDfs = new SequentialFixpointRuleController(false);
+        logicalRewrites.add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildConsolidationRuleCollection()));
+        logicalRewrites.add(new Pair<>(seqCtrlNoDfs, RuleCollections.buildPlanCleanupRuleCollection()));
+        return logicalRewrites;
+    }
+
     public static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> buildPhysical(
             ICcApplicationContext appCtx) {
         List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> defaultPhysicalRewrites = new ArrayList<>();
@@ -88,5 +106,4 @@
         defaultPhysicalRewrites.add(new Pair<>(seqOnceCtrl, RuleCollections.prepareForJobGenRuleCollection()));
         return defaultPhysicalRewrites;
     }
-
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java
index f625343..30c6623 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/ILangCompilationProvider.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.compiler.provider;
 
+import java.util.Set;
+
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
 import org.apache.asterix.algebra.base.ILangExtension;
 import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
@@ -54,4 +56,9 @@
      * @return the rule set factory of a language implementation
      */
     IRuleSetFactory getRuleSetFactory();
+
+    /**
+     * @return all configurable parameters of a language implementation.
+     */
+    Set<String> getCompilerOptions();
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/IRuleSetFactory.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/IRuleSetFactory.java
index 2300e4a..c643d21 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/IRuleSetFactory.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/IRuleSetFactory.java
@@ -21,24 +21,30 @@
 import java.util.List;
 
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
 
 public interface IRuleSetFactory {
 
+    enum RuleSetKind implements IRuleSetKind {
+        SAMPLING
+    }
+
     /**
      * @return the logical rewrites
-     * @throws AlgebricksException
      */
-    public List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getLogicalRewrites(
-            ICcApplicationContext appCtx) throws AlgebricksException;
+    List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getLogicalRewrites(ICcApplicationContext appCtx);
+
+    /**
+     * @return the logical rewrites of the specified kind
+     */
+    List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getLogicalRewrites(IRuleSetKind ruleSetKind,
+            ICcApplicationContext appCtx);
 
     /**
      * @return the physical rewrites
      */
-    public List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getPhysicalRewrites(
-            ICcApplicationContext appCtx) throws AlgebricksException;
-
+    List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> getPhysicalRewrites(ICcApplicationContext appCtx);
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
index 2c18d41..43963dc 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/compiler/provider/SqlppCompilationProvider.java
@@ -18,14 +18,27 @@
  */
 package org.apache.asterix.compiler.provider;
 
+import java.util.HashSet;
+import java.util.Set;
+
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
 import org.apache.asterix.algebra.base.ILangExtension;
+import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.external.feed.watch.FeedActivityDetails;
 import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
 import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.IRewriterFactory;
+import org.apache.asterix.lang.common.statement.StartFeedStatement;
+import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.lang.sqlpp.parser.SqlppParserFactory;
+import org.apache.asterix.lang.sqlpp.rewrites.SqlppQueryRewriter;
 import org.apache.asterix.lang.sqlpp.rewrites.SqlppRewriterFactory;
 import org.apache.asterix.lang.sqlpp.visitor.SqlppAstPrintVisitorFactory;
+import org.apache.asterix.optimizer.base.FuzzyUtils;
+import org.apache.asterix.optimizer.rules.DisjunctivePredicateToJoinRule;
+import org.apache.asterix.optimizer.rules.SetAsterixPhysicalOperatorsRule;
+import org.apache.asterix.optimizer.rules.util.EquivalenceClassUtils;
+import org.apache.asterix.translator.SqlppExpressionToPlanTranslator;
 import org.apache.asterix.translator.SqlppExpressionToPlanTranslatorFactory;
 
 public class SqlppCompilationProvider implements ILangCompilationProvider {
@@ -59,4 +72,24 @@
     public IRuleSetFactory getRuleSetFactory() {
         return new DefaultRuleSetFactory();
     }
+
+    @Override
+    public Set<String> getCompilerOptions() {
+        return new HashSet<>(Set.of(CompilerProperties.COMPILER_JOINMEMORY_KEY,
+                CompilerProperties.COMPILER_GROUPMEMORY_KEY, CompilerProperties.COMPILER_SORTMEMORY_KEY,
+                CompilerProperties.COMPILER_WINDOWMEMORY_KEY, CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
+                CompilerProperties.COMPILER_PARALLELISM_KEY, CompilerProperties.COMPILER_SORT_PARALLEL_KEY,
+                CompilerProperties.COMPILER_SORT_SAMPLES_KEY, CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY,
+                CompilerProperties.COMPILER_INDEXONLY_KEY, CompilerProperties.COMPILER_INTERNAL_SANITYCHECK_KEY,
+                CompilerProperties.COMPILER_EXTERNAL_FIELD_PUSHDOWN_KEY, CompilerProperties.COMPILER_SUBPLAN_MERGE_KEY,
+                CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY, CompilerProperties.COMPILER_ARRAYINDEX_KEY,
+                CompilerProperties.COMPILER_MIN_MEMORY_ALLOCATION_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
+                FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
+                StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
+                FeedActivityDetails.COLLECT_LOCATIONS, SqlppQueryRewriter.INLINE_WITH_OPTION,
+                SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION, "hash_merge", "output-record-type",
+                DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION,
+                SetAsterixPhysicalOperatorsRule.REWRITE_ATTEMPT_BATCH_ASSIGN,
+                EquivalenceClassUtils.REWRITE_INTERNAL_QUERYUID_PK, SqlppQueryRewriter.SQL_COMPAT_OPTION));
+    }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
index b1d0b47..b8685d5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
@@ -26,7 +26,6 @@
 import org.apache.asterix.external.library.ExternalFunctionDescriptorProvider;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IExternalFunctionInfo;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionManager;
 import org.apache.asterix.om.functions.IFunctionTypeInferer;
@@ -138,7 +137,7 @@
             throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
         IFunctionDescriptor fd;
-        if (expr.getFunctionInfo() instanceof IExternalFunctionInfo) {
+        if (expr.getFunctionInfo().isExternal()) {
             // Expr is an external function
             fd = ExternalFunctionDescriptorProvider.resolveExternalFunction(expr, env, context);
         } else {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
index 8b5c8bc..b6f79d9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
@@ -18,32 +18,70 @@
  */
 package org.apache.asterix.optimizer.base;
 
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.common.transactions.TxnId;
+import org.apache.asterix.common.utils.JobUtils;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.metadata.declared.DataSourceId;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.declared.ResultSetDataSink;
+import org.apache.asterix.metadata.declared.ResultSetSinkId;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.optimizer.rules.am.AccessMethodUtils;
+import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
+import org.apache.asterix.translator.ResultMetadata;
+import org.apache.asterix.translator.SessionConfig;
 import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.compiler.api.ICompiler;
+import org.apache.hyracks.algebricks.compiler.api.ICompilerFactory;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractDataSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
+import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.result.IResultSetReader;
+import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.control.nc.resources.memory.FrameManager;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleInputStream;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 
 public class AnalysisUtil {
+
+    private static final List<FunctionIdentifier> fieldAccessFunctions =
+            Arrays.asList(BuiltinFunctions.GET_DATA, BuiltinFunctions.GET_HANDLE, BuiltinFunctions.TYPE_OF);
+
     /*
      * If the first child of op is of type opType, then it returns that child,
      * o/w returns null.
@@ -204,11 +242,86 @@
         return true;
     }
 
-    private static List<FunctionIdentifier> fieldAccessFunctions = new ArrayList<>();
+    public static List<List<IAObject>> runQuery(Mutable<ILogicalOperator> topOp, List<LogicalVariable> resultVars,
+            IOptimizationContext queryOptCtx, IRuleSetKind ruleSetKind) throws AlgebricksException {
 
-    static {
-        fieldAccessFunctions.add(BuiltinFunctions.GET_DATA);
-        fieldAccessFunctions.add(BuiltinFunctions.GET_HANDLE);
-        fieldAccessFunctions.add(BuiltinFunctions.TYPE_OF);
+        MetadataProvider metadataProvider = (MetadataProvider) queryOptCtx.getMetadataProvider();
+        ICcApplicationContext appCtx = metadataProvider.getApplicationContext();
+        TxnId mainTxnId = metadataProvider.getTxnId();
+        try {
+            TxnId newTxnId = metadataProvider.getTxnIdFactory().create();
+            metadataProvider.setTxnId(newTxnId);
+
+            IVariableTypeEnvironment topOpTypeEnv = queryOptCtx.getOutputTypeEnvironment(topOp.getValue());
+            SerializerDeserializerProvider serdeProvider = SerializerDeserializerProvider.INSTANCE;
+
+            int nFields = resultVars.size();
+            List<Mutable<ILogicalExpression>> resultExprList = new ArrayList<>(nFields);
+            List<ISerializerDeserializer<?>> resultSerdeList = new ArrayList<>(nFields);
+
+            for (LogicalVariable var : resultVars) {
+                Object varType = topOpTypeEnv.getVarType(var);
+                if (varType == null) {
+                    throw new IllegalArgumentException("Cannot determine type of " + var);
+                }
+                resultSerdeList.add(serdeProvider.getSerializerDeserializer(varType));
+                resultExprList.add(new MutableObject<>(new VariableReferenceExpression(var)));
+            }
+
+            ResultMetadata resultMetadata = new ResultMetadata(SessionConfig.OutputFormat.ADM);
+            ResultSetId resultSetId = new ResultSetId(metadataProvider.getResultSetIdCounter().getAndInc());
+            ResultSetSinkId rssId = new ResultSetSinkId(resultSetId);
+            ResultSetDataSink sink = new ResultSetDataSink(rssId, null);
+
+            DistributeResultOperator resultOp = new DistributeResultOperator(resultExprList, sink, resultMetadata);
+            resultOp.getInputs().add(topOp);
+            queryOptCtx.computeAndSetTypeEnvironmentForOperator(resultOp);
+
+            MutableObject<ILogicalOperator> newResultOpRef = new MutableObject<>(resultOp);
+
+            ICompilerFactory compilerFactory = (ICompilerFactory) queryOptCtx.getCompilerFactory();
+            ICompiler compiler =
+                    compilerFactory.createCompiler(new ALogicalPlanImpl(newResultOpRef), queryOptCtx, ruleSetKind);
+            compiler.optimize();
+
+            JobSpecification jobSpec = compiler.createJob(appCtx, new JobEventListenerFactory(newTxnId, false));
+
+            JobId jobId = JobUtils.runJob(appCtx.getHcc(), jobSpec, true);
+
+            IResultSetReader resultSetReader = appCtx.getResultSet().createReader(jobId, resultSetId);
+            FrameManager frameManager = new FrameManager(queryOptCtx.getPhysicalOptimizationConfig().getFrameSize());
+            IFrame frame = new VSizeFrame(frameManager);
+
+            FrameTupleAccessor fta = new FrameTupleAccessor(null);
+            ByteArrayAccessibleInputStream bais = new ByteArrayAccessibleInputStream(frame.getBuffer().array(), 0, 0);
+            DataInputStream dis = new DataInputStream(bais);
+            List<List<IAObject>> result = new ArrayList<>();
+
+            while (resultSetReader.read(frame) > 0) {
+                ByteBuffer buffer = frame.getBuffer();
+                fta.reset(buffer);
+                int nTuples = fta.getTupleCount();
+                for (int tupleIdx = 0; tupleIdx < nTuples; tupleIdx++) {
+                    int tupleStart = fta.getTupleStartOffset(tupleIdx);
+                    int tupleEnd = fta.getTupleEndOffset(tupleIdx);
+                    bais.setContent(buffer.array(), tupleStart, tupleEnd - tupleStart);
+
+                    List<IAObject> values = new ArrayList<>(nFields);
+                    for (int fieldIdx = 0; fieldIdx < nFields; fieldIdx++) {
+                        IAObject value = (IAObject) resultSerdeList.get(fieldIdx).deserialize(dis);
+                        values.add(value);
+                    }
+                    result.add(values);
+                }
+            }
+
+            return result;
+        } catch (AlgebricksException e) {
+            throw e;
+        } catch (Exception e) {
+            throw new AlgebricksException(e);
+        } finally {
+            metadataProvider.setTxnId(mainTxnId);
+        }
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java
index 51c03eb..7b2dd7a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AsterixOptimizationContext.java
@@ -31,6 +31,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.IMissableTypeComputer;
 import org.apache.hyracks.algebricks.core.algebra.prettyprint.IPlanPrettyPrinter;
 import org.apache.hyracks.algebricks.core.rewriter.base.AlgebricksOptimizationContext;
+import org.apache.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 
@@ -39,17 +40,24 @@
 
 public final class AsterixOptimizationContext extends AlgebricksOptimizationContext {
 
-    private final Int2ObjectMap<Set<DataSource>> dataSourceMap = new Int2ObjectOpenHashMap<>();
+    private final Int2ObjectOpenHashMap<Set<DataSource>> dataSourceMap;
 
-    public AsterixOptimizationContext(int varCounter, IExpressionEvalSizeComputer expressionEvalSizeComputer,
+    public AsterixOptimizationContext(IOptimizationContextFactory optContextFactory, int varCounter,
+            IExpressionEvalSizeComputer expressionEvalSizeComputer,
             IMergeAggregationExpressionFactory mergeAggregationExpressionFactory,
             IExpressionTypeComputer expressionTypeComputer, IMissableTypeComputer nullableTypeComputer,
             IConflictingTypeResolver conflictingTypeResovler, PhysicalOptimizationConfig physicalOptimizationConfig,
             AlgebricksPartitionConstraint clusterLocations, IPlanPrettyPrinter prettyPrinter,
             IWarningCollector warningCollector) {
-        super(varCounter, expressionEvalSizeComputer, mergeAggregationExpressionFactory, expressionTypeComputer,
-                nullableTypeComputer, conflictingTypeResovler, physicalOptimizationConfig, clusterLocations,
-                prettyPrinter, warningCollector);
+        super(optContextFactory, varCounter, expressionEvalSizeComputer, mergeAggregationExpressionFactory,
+                expressionTypeComputer, nullableTypeComputer, conflictingTypeResovler, physicalOptimizationConfig,
+                clusterLocations, prettyPrinter, warningCollector);
+        dataSourceMap = new Int2ObjectOpenHashMap<>();
+    }
+
+    public AsterixOptimizationContext(AsterixOptimizationContext from) {
+        super(from);
+        dataSourceMap = from.dataSourceMap.clone();
     }
 
     public void addDataSource(DataSource dataSource) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index 10aedc7..7202f74 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -29,7 +29,6 @@
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.NoOpWarningCollector;
 import org.apache.asterix.common.exceptions.WarningCollector;
 import org.apache.asterix.dataflow.data.common.ExpressionTypeComputer;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
@@ -48,7 +47,6 @@
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IExternalFunctionInfo;
 import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
@@ -90,11 +88,14 @@
 import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.serializer.ResultSerializerFactoryProvider;
+import org.apache.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
 import org.apache.hyracks.api.application.IServiceContext;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
 import org.apache.hyracks.api.exceptions.Warning;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
@@ -151,8 +152,9 @@
         jobGenCtx = new JobGenContext(null, metadataProvider, appCtx, SerializerDeserializerProvider.INSTANCE,
                 BinaryHashFunctionFactoryProvider.INSTANCE, BinaryHashFunctionFamilyProvider.INSTANCE,
                 BinaryComparatorFactoryProvider.INSTANCE, TypeTraitProvider.INSTANCE, BinaryBooleanInspector.FACTORY,
-                BinaryIntegerInspector.FACTORY, ADMPrinterFactoryProvider.INSTANCE, MissingWriterFactory.INSTANCE,
-                NullWriterFactory.INSTANCE, UnnestingPositionWriterFactory.INSTANCE, null,
+                BinaryIntegerInspector.FACTORY, ADMPrinterFactoryProvider.INSTANCE, PrinterBasedWriterFactory.INSTANCE,
+                ResultSerializerFactoryProvider.INSTANCE, MissingWriterFactory.INSTANCE, NullWriterFactory.INSTANCE,
+                UnnestingPositionWriterFactory.INSTANCE, null,
                 new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())),
                 ExpressionTypeComputer.INSTANCE, null, null, null, null, GlobalConfig.DEFAULT_FRAME_SIZE, null,
                 NoOpWarningCollector.INSTANCE, 0, new PhysicalOptimizationConfig());
@@ -390,7 +392,7 @@
         private boolean canConstantFold(ScalarFunctionCallExpression function) throws AlgebricksException {
             // skip external functions because they're not available at compile time (on CC)
             IFunctionInfo fi = function.getFunctionInfo();
-            if (fi instanceof IExternalFunctionInfo) {
+            if (fi.isExternal()) {
                 return false;
             }
             // skip all functions that would produce records/arrays/multisets (derived types) in their open format
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index 3fc178a..09ad4d1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -27,6 +27,7 @@
 import java.util.Objects;
 import java.util.Set;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 import org.apache.asterix.algebra.operators.CommitOperator;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -190,18 +191,21 @@
             metaType = (ARecordType) mp.findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
         }
         List<Index> indexes = mp.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+        Stream<Index> indexStream = indexes.stream();
+        indexStream = indexStream.filter(index -> index.getIndexType() != IndexType.SAMPLE);
         if (primaryIndexModificationOp.getOperation() == Kind.INSERT && !primaryIndexModificationOp.isBulkload()) {
             // for insert, primary key index is handled together when primary index
-            indexes = indexes.stream().filter(index -> !index.isPrimaryKeyIndex()).collect(Collectors.toList());
+            indexStream = indexStream.filter(index -> !index.isPrimaryKeyIndex());
         }
-
-        // Set the top operator pointer to the primary IndexInsertDeleteOperator
-        ILogicalOperator currentTop = primaryIndexModificationOp;
+        indexes = indexStream.collect(Collectors.toList());
 
         // Put an n-gram or a keyword index in the later stage of index-update,
         // since TokenizeOperator needs to be involved.
         Collections.sort(indexes, (o1, o2) -> o1.getIndexType().ordinal() - o2.getIndexType().ordinal());
 
+        // Set the top operator pointer to the primary IndexInsertDeleteOperator
+        ILogicalOperator currentTop = primaryIndexModificationOp;
+
         // At this point, we have the data type info, and the indexes info as well
         int secondaryIndexTotalCnt = indexes.size() - 1;
         if (secondaryIndexTotalCnt > 0) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
index aa2b8fd..036c456 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
@@ -95,7 +95,8 @@
             if (dataSource.getDatasourceType() != DataSource.Type.EXTERNAL_DATASET
                     && dataSource.getDatasourceType() != DataSource.Type.INTERNAL_DATASET
                     && dataSource.getDatasourceType() != DataSource.Type.LOADABLE
-                    && dataSource.getDatasourceType() != DataSource.Type.FUNCTION) {
+                    && dataSource.getDatasourceType() != DataSource.Type.FUNCTION
+                    && dataSource.getDatasourceType() != DataSource.Type.SAMPLE) {
                 IMutationDataSource mds = (IMutationDataSource) dataSource;
                 if (mds.isChange()) {
                     transformers = new ArrayList<>();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index 7f1ff4a..5be9d66 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -821,7 +821,7 @@
         // Remember matching subtree.
         optFuncExpr.setOptimizableSubTree(funcVarIndex, subTree);
         List<String> fieldName = null;
-        MutableInt fieldSource = new MutableInt(0);
+        int fieldSource = 0;
         if (subTree.getDataSourceType() == DataSourceType.COLLECTION_SCAN) {
             ILogicalExpression expr = optFuncExpr.getArgument(funcVarIndex).getValue();
             if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
@@ -832,9 +832,11 @@
             if (subTree.getDataSourceType() == DataSourceType.DATASOURCE_SCAN) {
                 subTree.setLastMatchedDataSourceVars(0, funcVarIndex);
             }
-            fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0,
-                    subTree.getRecordType(), funcVarIndex, optFuncExpr.getArgument(funcVarIndex).getValue(),
-                    subTree.getMetaRecordType(), datasetMetaVar, fieldSource, false);
+            Pair<List<String>, Integer> fieldNameAndSource =
+                    AccessMethodUtils.getFieldNameSetStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0,
+                            funcVarIndex, optFuncExpr.getArgument(funcVarIndex).getValue(), context);
+            fieldName = fieldNameAndSource.first;
+            fieldSource = fieldNameAndSource.second;
             if (fieldName.isEmpty()) {
                 return;
             }
@@ -843,13 +845,13 @@
                 (IAType) context.getOutputTypeEnvironment(unnestOp).getType(optFuncExpr.getLogicalExpr(funcVarIndex));
         // Set the fieldName in the corresponding matched function
         // expression.
-        optFuncExpr.setFieldName(funcVarIndex, fieldName, fieldSource.intValue());
+        optFuncExpr.setFieldName(funcVarIndex, fieldName, fieldSource);
         optFuncExpr.setFieldType(funcVarIndex, fieldType);
 
         setTypeTag(context, subTree, optFuncExpr, funcVarIndex);
         if (subTree.hasDataSource()) {
             fillIndexExprs(datasetIndexes, fieldName, fieldType, optFuncExpr, optFuncExprIndex, funcVarIndex, subTree,
-                    analysisCtx, fieldSource.intValue(), accessMethod);
+                    analysisCtx, fieldSource, accessMethod);
         }
     }
 
@@ -860,7 +862,6 @@
         boolean doesArrayIndexQualify = context.getPhysicalOptimizationConfig().isArrayIndexEnabled()
                 && datasetIndexes.stream().anyMatch(i -> i.getIndexType() == IndexType.ARRAY);
         List<LogicalVariable> varList = assignOp.getVariables();
-        MutableInt fieldSource = new MutableInt(0);
         for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
             LogicalVariable var = varList.get(varIndex);
             int optVarIndex = optFuncExpr.findLogicalVar(var);
@@ -871,8 +872,7 @@
                             .analyzeVarForArrayIndexes(datasetIndexes, optFuncExpr, subTree, context, var, analysisCtx);
                     if (fieldTriplet != null && subTree.hasDataSource()) {
                         fillIndexExprs(datasetIndexes, fieldTriplet.second, fieldTriplet.third, optFuncExpr,
-                                optFuncExprIndex, fieldTriplet.first, subTree, analysisCtx, fieldSource.intValue(),
-                                accessMethod);
+                                optFuncExprIndex, fieldTriplet.first, subTree, analysisCtx, 0, accessMethod);
                     }
                 }
                 continue;
@@ -885,22 +885,22 @@
                 subTree.setLastMatchedDataSourceVars(varIndex, optVarIndex);
             }
 
-            fieldSource.setValue(0);
-            List<String> fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(optFuncExpr, subTree,
-                    assignOrUnnestIndex, varIndex, subTree.getRecordType(), optVarIndex,
-                    optFuncExpr.getArgument(optVarIndex).getValue(), subTree.getMetaRecordType(), datasetMetaVar,
-                    fieldSource, false);
+            Pair<List<String>, Integer> fieldNameAndSource =
+                    AccessMethodUtils.getFieldNameSetStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex,
+                            varIndex, optVarIndex, optFuncExpr.getArgument(optVarIndex).getValue(), context);
+            List<String> fieldName = fieldNameAndSource.first;
+            int fieldSource = fieldNameAndSource.second;
 
             IAType fieldType = (IAType) context.getOutputTypeEnvironment(assignOp).getVarType(var);
             // Set the fieldName in the corresponding matched
             // function expression.
-            optFuncExpr.setFieldName(optVarIndex, fieldName, fieldSource.intValue());
+            optFuncExpr.setFieldName(optVarIndex, fieldName, fieldSource);
             optFuncExpr.setFieldType(optVarIndex, fieldType);
 
             setTypeTag(context, subTree, optFuncExpr, optVarIndex);
             if (subTree.hasDataSource()) {
                 fillIndexExprs(datasetIndexes, fieldName, fieldType, optFuncExpr, optFuncExprIndex, optVarIndex,
-                        subTree, analysisCtx, fieldSource.intValue(), accessMethod);
+                        subTree, analysisCtx, fieldSource, accessMethod);
             }
         }
     }
@@ -989,16 +989,8 @@
     /**
      * Finds the field name of each variable in the ASSIGN or UNNEST operators of the sub-tree.
      */
-    protected void fillFieldNamesInTheSubTree(OptimizableOperatorSubTree subTree) throws AlgebricksException {
-        LogicalVariable datasetMetaVar = null;
-        if (subTree.getDataSourceType() != DataSourceType.COLLECTION_SCAN
-                && subTree.getDataSourceType() != DataSourceType.INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP) {
-            List<LogicalVariable> datasetVars = subTree.getDataSourceVariables();
-            if (subTree.getDataset().hasMetaPart()) {
-                datasetMetaVar = datasetVars.get(datasetVars.size() - 1);
-            }
-        }
-        MutableInt fieldSource = new MutableInt(0);
+    protected void fillFieldNamesInTheSubTree(OptimizableOperatorSubTree subTree, IOptimizationContext context)
+            throws AlgebricksException {
         for (int assignOrUnnestIndex = 0; assignOrUnnestIndex < subTree.getAssignsAndUnnests()
                 .size(); assignOrUnnestIndex++) {
             AbstractLogicalOperator op = subTree.getAssignsAndUnnests().get(assignOrUnnestIndex);
@@ -1009,10 +1001,8 @@
                     LogicalVariable var = varList.get(varIndex);
                     // funcVarIndex is not required. Thus, we set it to -1.
                     // optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
-                    fieldSource.setValue(0);
                     List<String> fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(null, subTree,
-                            assignOrUnnestIndex, varIndex, subTree.getRecordType(), -1, null,
-                            subTree.getMetaRecordType(), datasetMetaVar, fieldSource, false);
+                            assignOrUnnestIndex, varIndex, -1, null, context).first;
                     if (fieldName != null && !fieldName.isEmpty()) {
                         subTree.getVarsToFieldNameMap().put(var, fieldName);
                     }
@@ -1020,14 +1010,11 @@
             } else if (op.getOperatorTag() == LogicalOperatorTag.UNNEST) {
                 UnnestOperator unnestOp = (UnnestOperator) op;
                 LogicalVariable var = unnestOp.getVariable();
-                List<String> fieldName = null;
                 if (subTree.getDataSourceType() != DataSourceType.COLLECTION_SCAN) {
                     // funcVarIndex is not required. Thus, we set it to -1.
                     // optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
-                    fieldSource.setValue(0);
-                    fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(null, subTree, assignOrUnnestIndex, 0,
-                            subTree.getRecordType(), -1, null, subTree.getMetaRecordType(), datasetMetaVar, fieldSource,
-                            false);
+                    List<String> fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(null, subTree,
+                            assignOrUnnestIndex, 0, -1, null, context).first;
                     if (fieldName != null && !fieldName.isEmpty()) {
                         subTree.getVarsToFieldNameMap().put(var, fieldName);
                     }
@@ -1052,10 +1039,8 @@
                     LogicalVariable var = varList.get(varIndex);
                     // funcVarIndex is not required. Thus, we set it to -1.
                     // optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
-                    fieldSource.setValue(0);
                     List<String> fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(null, subTree,
-                            assignOrUnnestIndex, varIndex, subTree.getRecordType(), -1, null,
-                            subTree.getMetaRecordType(), datasetMetaVar, fieldSource, false);
+                            assignOrUnnestIndex, varIndex, -1, null, context).first;
                     if (fieldName != null && !fieldName.isEmpty()) {
                         subTree.getVarsToFieldNameMap().put(var, fieldName);
                     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
index 3b39d9d..ddc4964 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -24,7 +24,6 @@
 import static org.apache.asterix.om.functions.BuiltinFunctions.FIELD_ACCESS_NESTED;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
@@ -147,6 +146,8 @@
             BuiltinFunctions.YEAR_MONTH_DURATION_DEFAULT_NULL_CONSTRUCTOR,
             BuiltinFunctions.UUID_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.BINARY_BASE64_DEFAULT_NULL_CONSTRUCTOR);
 
+    private final static Pair<List<String>, Integer> NO_FIELD_NAME = new Pair<>(Collections.emptyList(), 0);
+
     public static void appendPrimaryIndexTypes(Dataset dataset, IAType itemType, IAType metaItemType,
             List<Object> target) throws AlgebricksException {
         ARecordType recordType = (ARecordType) itemType;
@@ -2898,17 +2899,16 @@
         return ann == null ? null : ann.getIndexNames();
     }
 
-    public static List<String> getFieldNameSetStepsFromSubTree(IOptimizableFuncExpr optFuncExpr,
-            OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, ARecordType recordType,
-            int funcVarIndex, ILogicalExpression parentFuncExpr, ARecordType metaType, LogicalVariable metaVar,
-            MutableInt fieldSource, boolean isUnnestOverVarAllowed) throws AlgebricksException {
+    public static Pair<List<String>, Integer> getFieldNameSetStepsFromSubTree(IOptimizableFuncExpr optFuncExpr,
+            OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, int funcVarIndex,
+            ILogicalExpression parentFuncExpr, IOptimizationContext context) throws AlgebricksException {
         if (optFuncExpr != null) {
             if (parentFuncExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                 optFuncExpr.addStepExpr(funcVarIndex, ((AbstractFunctionCallExpression) parentFuncExpr));
             }
         }
-        return getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, opIndex, assignVarIndex, recordType, funcVarIndex,
-                parentFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+        return getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, opIndex, assignVarIndex, funcVarIndex,
+                parentFuncExpr, context);
     }
 
     /**
@@ -2918,10 +2918,9 @@
      *
      * @throws AlgebricksException
      */
-    private static List<String> getFieldNameAndStepsFromSubTree(IOptimizableFuncExpr optFuncExpr,
-            OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, ARecordType recordType,
-            int funcVarIndex, ILogicalExpression parentFuncExpr, ARecordType metaType, LogicalVariable metaVar,
-            MutableInt fieldSource, boolean isUnnestOverVarAllowed) throws AlgebricksException {
+    private static Pair<List<String>, Integer> getFieldNameAndStepsFromSubTree(IOptimizableFuncExpr optFuncExpr,
+            OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, int funcVarIndex,
+            ILogicalExpression parentFuncExpr, IOptimizationContext context) throws AlgebricksException {
         // Get expression corresponding to opVar at varIndex.
         AbstractLogicalExpression expr = null;
         AbstractFunctionCallExpression childFuncExpr = null;
@@ -2931,23 +2930,23 @@
             expr = (AbstractLogicalExpression) assignOp.getExpressions().get(assignVarIndex).getValue();
             // Can't get a field name from a constant expression. So, return null.
             if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-                return Collections.emptyList();
+                return NO_FIELD_NAME;
             }
             childFuncExpr = (AbstractFunctionCallExpression) expr;
         } else {
             UnnestOperator unnestOp = (UnnestOperator) op;
             expr = (AbstractLogicalExpression) unnestOp.getExpressionRef().getValue();
             if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-                return Collections.emptyList();
+                return NO_FIELD_NAME;
             }
             childFuncExpr = (AbstractFunctionCallExpression) expr;
             if (childFuncExpr.getFunctionIdentifier() != BuiltinFunctions.SCAN_COLLECTION) {
-                return Collections.emptyList();
+                return NO_FIELD_NAME;
             }
             expr = (AbstractLogicalExpression) childFuncExpr.getArguments().get(0).getValue();
         }
         if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-            return Collections.emptyList();
+            return NO_FIELD_NAME;
         }
         AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
         FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
@@ -2960,21 +2959,21 @@
         if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_NAME) {
             fieldName = ConstantExpressionUtil.getStringArgument(funcExpr, 1);
             if (fieldName == null) {
-                return Collections.emptyList();
+                return NO_FIELD_NAME;
             }
             isFieldAccess = true;
             isByName = true;
         } else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_INDEX) {
             Integer idx = ConstantExpressionUtil.getIntArgument(funcExpr, 1);
             if (idx == null) {
-                return Collections.emptyList();
+                return NO_FIELD_NAME;
             }
             fieldIndex = idx;
             isFieldAccess = true;
         } else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_NESTED) {
             ILogicalExpression nameArg = funcExpr.getArguments().get(1).getValue();
             if (nameArg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                return Collections.emptyList();
+                return NO_FIELD_NAME;
             }
             ConstantExpression constExpr = (ConstantExpression) nameArg;
             AOrderedList orderedNestedFieldName =
@@ -2989,11 +2988,6 @@
         if (isFieldAccess) {
             LogicalVariable sourceVar =
                     ((VariableReferenceExpression) funcExpr.getArguments().get(0).getValue()).getVariableReference();
-            if (sourceVar.equals(metaVar)) {
-                fieldSource.setValue(1);
-            } else {
-                fieldSource.setValue(0);
-            }
             if (optFuncExpr != null) {
                 optFuncExpr.setLogicalExpr(funcVarIndex, parentFuncExpr);
                 optFuncExpr.addStepExpr(funcVarIndex, funcExpr);
@@ -3024,8 +3018,7 @@
                     if (parentVars.contains(var)) {
                         //Found the variable we are looking for.
                         //return assign and index of expression
-                        int[] returnValues = { i, varIndex };
-                        assignAndExpressionIndexes = returnValues;
+                        assignAndExpressionIndexes = new int[] { i, varIndex };
                     }
                 }
             }
@@ -3033,34 +3026,25 @@
                 //We found the nested assign
 
                 //Recursive call on nested assign
-                List<String> parentFieldNames = getFieldNameAndStepsFromSubTree(optFuncExpr, subTree,
-                        assignAndExpressionIndexes[0], assignAndExpressionIndexes[1], recordType, funcVarIndex,
-                        parentFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+                Pair<List<String>, Integer> parentFieldNames =
+                        getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignAndExpressionIndexes[0],
+                                assignAndExpressionIndexes[1], funcVarIndex, parentFuncExpr, context);
 
-                boolean isPreviousOperatorLegalUnnest = isUnnestOverVarAllowed && subTree.getAssignsAndUnnests()
-                        .get(assignAndExpressionIndexes[0]).getOperatorTag().equals(LogicalOperatorTag.UNNEST);
-                if (parentFieldNames.isEmpty() && !isPreviousOperatorLegalUnnest) {
+                if (parentFieldNames.first.isEmpty()) {
                     //Nested assign was not a field access.
                     //We will not use index
-                    return Collections.emptyList();
-                } else if (isPreviousOperatorLegalUnnest) {
-                    parentFieldNames = new ArrayList<>();
+                    return NO_FIELD_NAME;
                 }
 
                 if (!isByName) {
-                    IAType subFieldType;
-                    if (isUnnestOverVarAllowed && isPreviousOperatorLegalUnnest) {
-                        // In the case of UNNESTing over a variable, we use the record type given by our caller instead.
-                        subFieldType = sourceVar.equals(metaVar) ? metaType : recordType;
-                    } else {
-                        subFieldType = sourceVar.equals(metaVar) ? metaType.getSubFieldType(parentFieldNames)
-                                : recordType.getSubFieldType(parentFieldNames);
-                        // Sub-field type can be AUnionType in case if optional. Thus, needs to get the actual type.
-                        subFieldType = TypeComputeUtils.getActualType(subFieldType);
-                        if (subFieldType.getTypeTag() != ATypeTag.OBJECT) {
-                            throw CompilationException.create(ErrorCode.TYPE_CONVERT, subFieldType,
-                                    ARecordType.class.getName());
-                        }
+                    IVariableTypeEnvironment outputTypeEnvironment = context.getOutputTypeEnvironment(
+                            subTree.getAssignsAndUnnests().get(assignAndExpressionIndexes[0]));
+                    IAType subFieldType = (IAType) outputTypeEnvironment.getVarType(sourceVar);
+                    // Sub-field type can be AUnionType in case if optional. Thus, needs to get the actual type.
+                    subFieldType = TypeComputeUtils.getActualType(subFieldType);
+                    if (subFieldType.getTypeTag() != ATypeTag.OBJECT) {
+                        throw CompilationException.create(ErrorCode.TYPE_CONVERT, subFieldType,
+                                ARecordType.class.getName());
                     }
                     fieldName = ((ARecordType) subFieldType).getFieldNames()[fieldIndex];
 
@@ -3070,11 +3054,9 @@
                 }
                 //add fieldName to the nested fieldName, return
                 if (nestedAccessFieldName != null) {
-                    for (int i = 0; i < nestedAccessFieldName.size(); i++) {
-                        parentFieldNames.add(nestedAccessFieldName.get(i));
-                    }
+                    parentFieldNames.first.addAll(nestedAccessFieldName);
                 } else {
-                    parentFieldNames.add(fieldName);
+                    parentFieldNames.first.add(fieldName);
                 }
                 return (parentFieldNames);
             }
@@ -3083,15 +3065,15 @@
                 optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
             }
             //no nested assign, we are at the lowest level.
+            OptimizableOperatorSubTree.RecordTypeSource recType = subTree.getRecordTypeFor(sourceVar);
             if (isByName) {
                 if (nestedAccessFieldName != null) {
-                    return nestedAccessFieldName;
+                    return new Pair<>(nestedAccessFieldName, recType.sourceIndicator);
                 }
-                return new ArrayList<>(Arrays.asList(fieldName));
+                return new Pair<>(new ArrayList<>(List.of(fieldName)), recType.sourceIndicator);
             }
-            return new ArrayList<>(Arrays.asList(sourceVar.equals(metaVar) ? metaType.getFieldNames()[fieldIndex]
-                    : recordType.getFieldNames()[fieldIndex]));
-
+            return new Pair<>(new ArrayList<>(List.of(recType.recordType.getFieldNames()[fieldIndex])),
+                    recType.sourceIndicator);
         }
 
         // We use a part of the field in edit distance computation
@@ -3101,17 +3083,17 @@
         }
         List<Mutable<ILogicalExpression>> funcArgs = funcExpr.getArguments();
         if (funcArgs.isEmpty()) {
-            return Collections.emptyList();
+            return NO_FIELD_NAME;
         }
         // We expect the function's argument to be a variable, otherwise we
         // cannot apply an index.
         ILogicalExpression argExpr = funcArgs.get(0).getValue();
         if (argExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
-            return Collections.emptyList();
+            return NO_FIELD_NAME;
         }
         for (int i = 1; i < funcArgs.size(); i++) {
             if (funcArgs.get(i).getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                return Collections.emptyList();
+                return NO_FIELD_NAME;
             }
         }
         if (optFuncExpr != null) {
@@ -3131,20 +3113,19 @@
                     if (var.equals(curVar) && optFuncExpr != null) {
                         optFuncExpr.setSourceVar(funcVarIndex, var);
                         return getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, varIndex,
-                                recordType, funcVarIndex, childFuncExpr, metaType, metaVar, fieldSource,
-                                isUnnestOverVarAllowed);
+                                funcVarIndex, childFuncExpr, context);
                     }
                 }
             } else {
                 UnnestOperator unnestOp = (UnnestOperator) curOp;
                 LogicalVariable var = unnestOp.getVariable();
                 if (var.equals(curVar)) {
-                    getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, recordType,
-                            funcVarIndex, childFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+                    getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, funcVarIndex,
+                            childFuncExpr, context);
                 }
             }
         }
-        return Collections.emptyList();
+        return NO_FIELD_NAME;
     }
 
     public static Triple<Integer, List<String>, IAType> analyzeVarForArrayIndexes(List<Index> datasetIndexes,
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
index 6f53219..751e216 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
@@ -382,10 +382,10 @@
                     // Finds the field name of each variable in the sub-tree such as variables for order by.
                     // This step is required when checking index-only plan.
                     if (checkLeftSubTreeMetadata) {
-                        fillFieldNamesInTheSubTree(leftSubTree);
+                        fillFieldNamesInTheSubTree(leftSubTree, context);
                     }
                     if (checkRightSubTreeMetadata) {
-                        fillFieldNamesInTheSubTree(rightSubTree);
+                        fillFieldNamesInTheSubTree(rightSubTree, context);
                     }
 
                     // Applies the plan transformation using chosen index.
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index d9b5da9..1cf4b0c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -453,7 +453,7 @@
                     AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenIndexes.get(0).first);
 
                     // Finds the field name of each variable in the sub-tree.
-                    fillFieldNamesInTheSubTree(subTree);
+                    fillFieldNamesInTheSubTree(subTree, context);
 
                     // Finally, try to apply plan transformation using chosen index.
                     res = chosenIndexes.get(0).first.applySelectPlanTransformation(afterSelectRefs, selectRef, subTree,
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index c55edf9..bbfb365 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -59,7 +59,7 @@
  */
 public class OptimizableOperatorSubTree {
 
-    public static enum DataSourceType {
+    public enum DataSourceType {
         DATASOURCE_SCAN,
         EXTERNAL_SCAN,
         PRIMARY_INDEX_LOOKUP,
@@ -68,6 +68,16 @@
         NO_DATASOURCE
     }
 
+    public static class RecordTypeSource {
+        final ARecordType recordType;
+        final int sourceIndicator;
+
+        RecordTypeSource(ARecordType recordType, int sourceIndicator) {
+            this.recordType = recordType;
+            this.sourceIndicator = sourceIndicator;
+        }
+    }
+
     private ILogicalOperator root = null;
     private Mutable<ILogicalOperator> rootRef = null;
     private final List<Mutable<ILogicalOperator>> assignsAndUnnestsRefs = new ArrayList<>();
@@ -91,6 +101,7 @@
     private List<DataSourceType> ixJoinOuterAdditionalDataSourceTypes = null;
     private List<Dataset> ixJoinOuterAdditionalDatasets = null;
     private List<ARecordType> ixJoinOuterAdditionalRecordTypes = null;
+    private final Map<LogicalVariable, RecordTypeSource> varsToRecordType = new HashMap<>();
 
     /**
      * Identifies the root of the subtree and initializes the data-source, assign, and unnest information.
@@ -172,10 +183,10 @@
                             AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
                             jobGenParams.readFromFuncArgs(f.getArguments());
                             if (jobGenParams.isPrimaryIndex()) {
-                                intializeDataSourceRefAndType(DataSourceType.PRIMARY_INDEX_LOOKUP, subTreeOpRef);
+                                initializeDataSourceRefAndType(DataSourceType.PRIMARY_INDEX_LOOKUP, subTreeOpRef);
                                 dataSourceFound = true;
                             } else if (unnestMapOp.getGenerateCallBackProceedResultVar()) {
-                                intializeDataSourceRefAndType(DataSourceType.INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP,
+                                initializeDataSourceRefAndType(DataSourceType.INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP,
                                         subTreeOpRef);
                                 dataSourceFound = true;
                             }
@@ -221,7 +232,7 @@
         return false;
     }
 
-    private void intializeDataSourceRefAndType(DataSourceType dsType, Mutable<ILogicalOperator> opRef) {
+    private void initializeDataSourceRefAndType(DataSourceType dsType, Mutable<ILogicalOperator> opRef) {
         if (getDataSourceRef() == null) {
             setDataSourceRef(opRef);
             setDataSourceType(dsType);
@@ -238,12 +249,6 @@
      * Also sets recordType to be the type of that dataset.
      */
     public boolean setDatasetAndTypeMetadata(MetadataProvider metadataProvider) throws AlgebricksException {
-        DataverseName dataverseName = null;
-        String datasetName = null;
-
-        Dataset ds = null;
-        ARecordType rType = null;
-
         List<Mutable<ILogicalOperator>> sourceOpRefs = new ArrayList<>();
         List<DataSourceType> dsTypes = new ArrayList<>();
 
@@ -259,6 +264,9 @@
         }
 
         for (int i = 0; i < sourceOpRefs.size(); i++) {
+            List<LogicalVariable> vars;
+            DataverseName dataverseName;
+            String datasetName;
             switch (dsTypes.get(i)) {
                 case DATASOURCE_SCAN:
                     DataSourceScanOperator dataSourceScan = (DataSourceScanOperator) sourceOpRefs.get(i).getValue();
@@ -272,6 +280,7 @@
                     Pair<DataverseName, String> datasetInfo = AnalysisUtil.getDatasetInfo(dataSourceScan);
                     dataverseName = datasetInfo.first;
                     datasetName = datasetInfo.second;
+                    vars = dataSourceScan.getScanVariables();
                     break;
                 case PRIMARY_INDEX_LOOKUP:
                 case INDEXONLY_PLAN_SECONDARY_INDEX_LOOKUP:
@@ -282,12 +291,14 @@
                     jobGenParams.readFromFuncArgs(f.getArguments());
                     datasetName = jobGenParams.getDatasetName();
                     dataverseName = jobGenParams.getDataverseName();
+                    vars = unnestMapOp.getScanVariables();
                     break;
                 case EXTERNAL_SCAN:
                     UnnestMapOperator externalScan = (UnnestMapOperator) sourceOpRefs.get(i).getValue();
                     datasetInfo = AnalysisUtil.getExternalDatasetInfo(externalScan);
                     dataverseName = datasetInfo.first;
                     datasetName = datasetInfo.second;
+                    vars = externalScan.getScanVariables();
                     break;
                 case COLLECTION_SCAN:
                     if (i != 0) {
@@ -303,7 +314,7 @@
                 return false;
             }
             // Find the dataset corresponding to the datasource in the metadata.
-            ds = metadataProvider.findDataset(dataverseName, datasetName);
+            Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
             if (ds == null) {
                 throw new CompilationException(ErrorCode.NO_METADATA_FOR_DATASET, root.getSourceLocation(),
                         datasetName);
@@ -318,26 +329,31 @@
                     getIxJoinOuterAdditionalRecordTypes().add(null);
                 }
             }
-            rType = (ARecordType) itemType;
+            ARecordType rType = (ARecordType) itemType;
 
             // Get the meta record type for that dataset.
-            IAType metaItemType =
-                    metadataProvider.findType(ds.getMetaItemTypeDataverseName(), ds.getMetaItemTypeName());
+            ARecordType metaItemType = (ARecordType) metadataProvider.findType(ds.getMetaItemTypeDataverseName(),
+                    ds.getMetaItemTypeName());
 
             // First index is always the primary datasource in this subtree.
             if (i == 0) {
                 setDataset(ds);
                 setRecordType(rType);
-                setMetaRecordType((ARecordType) metaItemType);
+                setMetaRecordType(metaItemType);
             } else {
                 getIxJoinOuterAdditionalDatasets().add(ds);
                 getIxJoinOuterAdditionalRecordTypes().add(rType);
             }
 
-            dataverseName = null;
-            datasetName = null;
-            ds = null;
-            rType = null;
+            if (!vars.isEmpty()) {
+                int numVars = vars.size();
+                if (ds.hasMetaPart()) {
+                    varsToRecordType.put(vars.get(numVars - 2), new RecordTypeSource(rType, 0));
+                    varsToRecordType.put(vars.get(numVars - 1), new RecordTypeSource(metaItemType, 1));
+                } else {
+                    varsToRecordType.put(vars.get(numVars - 1), new RecordTypeSource(rType, 0));
+                }
+            }
         }
 
         return true;
@@ -364,17 +380,6 @@
         return getDataSourceType() == DataSourceType.DATASOURCE_SCAN;
     }
 
-    public boolean hasIxJoinOuterAdditionalDataSourceScan() {
-        if (getIxJoinOuterAdditionalDataSourceTypes() != null) {
-            for (int i = 0; i < getIxJoinOuterAdditionalDataSourceTypes().size(); i++) {
-                if (getIxJoinOuterAdditionalDataSourceTypes().get(i) == DataSourceType.DATASOURCE_SCAN) {
-                    return true;
-                }
-            }
-        }
-        return false;
-    }
-
     public void reset() {
         setRoot(null);
         setRootRef(null);
@@ -392,6 +397,7 @@
         setIxJoinOuterAdditionalRecordTypes(null);
         lastMatchedDataSourceVars.first = -1;
         lastMatchedDataSourceVars.second = -1;
+        varsToRecordType.clear();
     }
 
     /**
@@ -545,6 +551,10 @@
         return recordType;
     }
 
+    public RecordTypeSource getRecordTypeFor(LogicalVariable var) {
+        return varsToRecordType.get(var);
+    }
+
     public void setRecordType(ARecordType recordType) {
         this.recordType = recordType;
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
index ac7fabe..b0f9a1c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
@@ -40,6 +40,7 @@
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.statement.AnalyzeStatement;
 import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
 import org.apache.asterix.lang.common.statement.CreateDataverseStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedStatement;
@@ -371,6 +372,17 @@
                             dataverseName);
                 }
                 break;
+
+            case ANALYZE:
+                AnalyzeStatement analyzeStmt = (AnalyzeStatement) stmt;
+                if (analyzeStmt.getDataverseName() != null) {
+                    dataverseName = analyzeStmt.getDataverseName();
+                }
+                invalidOperation = isMetadataDataverse(dataverseName);
+                if (invalidOperation) {
+                    message = String.format(BAD_DATAVERSE_OBJECT_DDL_MESSAGE, "analyze", dataset(), dataverseName);
+                }
+                break;
         }
 
         if (invalidOperation) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/ResultMetadata.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultMetadata.java
similarity index 96%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/ResultMetadata.java
rename to asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultMetadata.java
index 94360a1..78f84ff 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/ResultMetadata.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ResultMetadata.java
@@ -16,12 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.api.common;
+package org.apache.asterix.translator;
 
 import java.util.List;
 import java.util.Set;
 
-import org.apache.asterix.translator.SessionConfig;
 import org.apache.hyracks.api.exceptions.Warning;
 import org.apache.hyracks.api.result.IResultMetadata;
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 3a9d54d..9b9dfac 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -40,6 +40,7 @@
 import org.apache.asterix.common.api.IResponsePrinter;
 import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.common.config.OptimizationConfUtil;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
@@ -54,7 +55,6 @@
 import org.apache.asterix.dataflow.data.common.MergeAggregationExpressionFactory;
 import org.apache.asterix.dataflow.data.common.MissableTypeComputer;
 import org.apache.asterix.dataflow.data.common.PartialAggregationTypeComputer;
-import org.apache.asterix.external.feed.watch.FeedActivityDetails;
 import org.apache.asterix.formats.base.IDataFormat;
 import org.apache.asterix.jobgen.QueryLogicalExpressionJobGen;
 import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
@@ -63,27 +63,18 @@
 import org.apache.asterix.lang.common.base.IRewriterFactory;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
-import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.Query;
-import org.apache.asterix.lang.common.statement.StartFeedStatement;
-import org.apache.asterix.lang.common.statement.ViewDecl;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
-import org.apache.asterix.lang.common.util.FunctionUtil;
-import org.apache.asterix.lang.sqlpp.rewrites.SqlppQueryRewriter;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.optimizer.base.AsterixOptimizationContext;
-import org.apache.asterix.optimizer.base.FuzzyUtils;
-import org.apache.asterix.optimizer.rules.DisjunctivePredicateToJoinRule;
-import org.apache.asterix.optimizer.rules.SetAsterixPhysicalOperatorsRule;
-import org.apache.asterix.optimizer.rules.util.EquivalenceClassUtils;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import org.apache.asterix.translator.ExecutionPlans;
 import org.apache.asterix.translator.IRequestParameters;
+import org.apache.asterix.translator.ResultMetadata;
 import org.apache.asterix.translator.SessionConfig;
 import org.apache.asterix.translator.SessionOutput;
-import org.apache.asterix.translator.SqlppExpressionToPlanTranslator;
 import org.apache.asterix.utils.ResourceUtils;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -106,6 +97,8 @@
 import org.apache.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.data.IPrinterFactoryProvider;
+import org.apache.hyracks.algebricks.runtime.serializer.ResultSerializerFactoryProvider;
+import org.apache.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
 import org.apache.hyracks.api.client.IClusterInfoCollector;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.client.NodeControllerInfo;
@@ -121,7 +114,6 @@
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectWriter;
-import com.google.common.collect.ImmutableSet;
 
 /**
  * Provides helper methods for compilation of a query into a JobSpec and submission
@@ -133,29 +125,11 @@
 
     public static final String PREFIX_INTERNAL_PARAMETERS = "_internal";
 
-    // A white list of supported configurable parameters.
-    private static final Set<String> CONFIGURABLE_PARAMETER_NAMES = ImmutableSet.of(
-            CompilerProperties.COMPILER_JOINMEMORY_KEY, CompilerProperties.COMPILER_GROUPMEMORY_KEY,
-            CompilerProperties.COMPILER_SORTMEMORY_KEY, CompilerProperties.COMPILER_WINDOWMEMORY_KEY,
-            CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY, CompilerProperties.COMPILER_PARALLELISM_KEY,
-            CompilerProperties.COMPILER_SORT_PARALLEL_KEY, CompilerProperties.COMPILER_SORT_SAMPLES_KEY,
-            CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY, CompilerProperties.COMPILER_INDEXONLY_KEY,
-            CompilerProperties.COMPILER_INTERNAL_SANITYCHECK_KEY,
-            CompilerProperties.COMPILER_EXTERNAL_FIELD_PUSHDOWN_KEY, CompilerProperties.COMPILER_SUBPLAN_MERGE_KEY,
-            CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY, CompilerProperties.COMPILER_ARRAYINDEX_KEY,
-            CompilerProperties.COMPILER_MIN_MEMORY_ALLOCATION_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
-            FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
-            StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
-            FeedActivityDetails.COLLECT_LOCATIONS, SqlppQueryRewriter.INLINE_WITH_OPTION,
-            SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION, "hash_merge", "output-record-type",
-            DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION,
-            SetAsterixPhysicalOperatorsRule.REWRITE_ATTEMPT_BATCH_ASSIGN,
-            EquivalenceClassUtils.REWRITE_INTERNAL_QUERYUID_PK, SqlppQueryRewriter.SQL_COMPAT_OPTION);
-
     private final IRewriterFactory rewriterFactory;
     private final IAstPrintVisitorFactory astPrintVisitorFactory;
     private final ILangExpressionToPlanTranslatorFactory translatorFactory;
     private final IRuleSetFactory ruleSetFactory;
+    private final Set<String> configurableParameterNames;
     private final ExecutionPlans executionPlans;
 
     public APIFramework(ILangCompilationProvider compilationProvider) {
@@ -163,6 +137,7 @@
         this.astPrintVisitorFactory = compilationProvider.getAstPrintVisitorFactory();
         this.translatorFactory = compilationProvider.getExpressionToPlanTranslatorFactory();
         this.ruleSetFactory = compilationProvider.getRuleSetFactory();
+        this.configurableParameterNames = compilationProvider.getCompilerOptions();
         executionPlans = new ExecutionPlans();
     }
 
@@ -181,17 +156,21 @@
                 IConflictingTypeResolver conflictingTypeResolver, PhysicalOptimizationConfig physicalOptimizationConfig,
                 AlgebricksPartitionConstraint clusterLocations, IWarningCollector warningCollector) {
             IPlanPrettyPrinter prettyPrinter = PlanPrettyPrinter.createStringPlanPrettyPrinter();
-            return new AsterixOptimizationContext(varCounter, expressionEvalSizeComputer,
+            return new AsterixOptimizationContext(this, varCounter, expressionEvalSizeComputer,
                     mergeAggregationExpressionFactory, expressionTypeComputer, missableTypeComputer,
                     conflictingTypeResolver, physicalOptimizationConfig, clusterLocations, prettyPrinter,
                     warningCollector);
         }
+
+        @Override
+        public IOptimizationContext cloneOptimizationContext(IOptimizationContext oc) {
+            return new AsterixOptimizationContext((AsterixOptimizationContext) oc);
+        }
     }
 
-    public Pair<IReturningStatement, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
-            List<ViewDecl> declaredViews, MetadataProvider metadataProvider, IReturningStatement q,
-            SessionOutput output, boolean allowNonStoredUdfCalls, boolean inlineUdfsAndViews,
-            Collection<VarIdentifier> externalVars, IWarningCollector warningCollector) throws CompilationException {
+    public Pair<IReturningStatement, Integer> reWriteQuery(LangRewritingContext langRewritingContext,
+            IReturningStatement q, SessionOutput output, boolean allowNonStoredUdfCalls, boolean inlineUdfsAndViews,
+            Collection<VarIdentifier> externalVars) throws CompilationException {
         if (q == null) {
             return null;
         }
@@ -200,9 +179,7 @@
             generateExpressionTree(q);
         }
         IQueryRewriter rw = rewriterFactory.createQueryRewriter();
-        LangRewritingContext rwCtx = new LangRewritingContext(metadataProvider, declaredFunctions, declaredViews,
-                warningCollector, q.getVarCounter());
-        rw.rewrite(rwCtx, q, allowNonStoredUdfCalls, inlineUdfsAndViews, externalVars);
+        rw.rewrite(langRewritingContext, q, allowNonStoredUdfCalls, inlineUdfsAndViews, externalVars);
         return new Pair<>(q, q.getVarCounter());
     }
 
@@ -237,7 +214,8 @@
                 && conf.is(SessionConfig.OOB_LOGICAL_PLAN)) {
             generateLogicalPlan(plan, output.config().getPlanFormat());
         }
-        CompilerProperties compilerProperties = metadataProvider.getApplicationContext().getCompilerProperties();
+        ICcApplicationContext ccAppContext = metadataProvider.getApplicationContext();
+        CompilerProperties compilerProperties = ccAppContext.getCompilerProperties();
         Map<String, Object> querySpecificConfig = validateConfig(metadataProvider.getConfig(), sourceLoc);
         final PhysicalOptimizationConfig physOptConf =
                 OptimizationConfUtil.createPhysicalOptimizationConf(compilerProperties, querySpecificConfig, sourceLoc);
@@ -245,8 +223,9 @@
         HeuristicCompilerFactoryBuilder builder =
                 new HeuristicCompilerFactoryBuilder(OptimizationContextFactory.INSTANCE);
         builder.setPhysicalOptimizationConfig(physOptConf);
-        builder.setLogicalRewrites(ruleSetFactory.getLogicalRewrites(metadataProvider.getApplicationContext()));
-        builder.setPhysicalRewrites(ruleSetFactory.getPhysicalRewrites(metadataProvider.getApplicationContext()));
+        builder.setLogicalRewrites(() -> ruleSetFactory.getLogicalRewrites(ccAppContext));
+        builder.setLogicalRewritesByKind(kind -> ruleSetFactory.getLogicalRewrites(kind, ccAppContext));
+        builder.setPhysicalRewrites(() -> ruleSetFactory.getPhysicalRewrites(ccAppContext));
         IDataFormat format = metadataProvider.getDataFormat();
         ICompilerFactory compilerFactory = builder.create();
         builder.setExpressionEvalSizeComputer(format.getExpressionEvalSizeComputer());
@@ -264,6 +243,24 @@
                 chooseLocations(clusterInfoCollector, parallelism, metadataProvider.getClusterLocations());
         builder.setClusterLocations(computationLocations);
 
+        builder.setBinaryBooleanInspectorFactory(format.getBinaryBooleanInspectorFactory());
+        builder.setBinaryIntegerInspectorFactory(format.getBinaryIntegerInspectorFactory());
+        builder.setComparatorFactoryProvider(format.getBinaryComparatorFactoryProvider());
+        builder.setExpressionRuntimeProvider(
+                new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())));
+        builder.setHashFunctionFactoryProvider(format.getBinaryHashFunctionFactoryProvider());
+        builder.setHashFunctionFamilyProvider(format.getBinaryHashFunctionFamilyProvider());
+        builder.setMissingWriterFactory(format.getMissingWriterFactory());
+        builder.setNullWriterFactory(format.getNullWriterFactory());
+        builder.setUnnestingPositionWriterFactory(format.getUnnestingPositionWriterFactory());
+        builder.setPredicateEvaluatorFactoryProvider(format.getPredicateEvaluatorFactoryProvider());
+        builder.setPrinterProvider(getPrinterFactoryProvider(format, conf.fmt()));
+        builder.setWriterFactory(PrinterBasedWriterFactory.INSTANCE);
+        builder.setResultSerializerFactoryProvider(ResultSerializerFactoryProvider.INSTANCE);
+        builder.setSerializerDeserializerProvider(format.getSerdeProvider());
+        builder.setTypeTraitProvider(format.getTypeTraitProvider());
+        builder.setNormalizedKeyComputerFactoryProvider(format.getNormalizedKeyComputerFactoryProvider());
+
         ICompiler compiler = compilerFactory.createCompiler(plan, metadataProvider, t.getVarCounter());
         if (conf.isOptimize()) {
             compiler.optimize();
@@ -312,32 +309,16 @@
             return null;
         }
 
-        builder.setBinaryBooleanInspectorFactory(format.getBinaryBooleanInspectorFactory());
-        builder.setBinaryIntegerInspectorFactory(format.getBinaryIntegerInspectorFactory());
-        builder.setComparatorFactoryProvider(format.getBinaryComparatorFactoryProvider());
-        builder.setExpressionRuntimeProvider(
-                new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())));
-        builder.setHashFunctionFactoryProvider(format.getBinaryHashFunctionFactoryProvider());
-        builder.setHashFunctionFamilyProvider(format.getBinaryHashFunctionFamilyProvider());
-        builder.setMissingWriterFactory(format.getMissingWriterFactory());
-        builder.setNullWriterFactory(format.getNullWriterFactory());
-        builder.setUnnestingPositionWriterFactory(format.getUnnestingPositionWriterFactory());
-        builder.setPredicateEvaluatorFactoryProvider(format.getPredicateEvaluatorFactoryProvider());
-        builder.setPrinterProvider(getPrinterFactoryProvider(format, conf.fmt()));
-        builder.setSerializerDeserializerProvider(format.getSerdeProvider());
-        builder.setTypeTraitProvider(format.getTypeTraitProvider());
-        builder.setNormalizedKeyComputerFactoryProvider(format.getNormalizedKeyComputerFactoryProvider());
-
         JobEventListenerFactory jobEventListenerFactory =
                 new JobEventListenerFactory(txnId, metadataProvider.isWriteTransaction());
-        JobSpecification spec = compiler.createJob(metadataProvider.getApplicationContext(), jobEventListenerFactory);
+        JobSpecification spec = compiler.createJob(ccAppContext, jobEventListenerFactory);
 
         if (isQuery) {
             if (requestParameters == null || !requestParameters.isSkipAdmissionPolicy()) {
                 // Sets a required capacity, only for read-only queries.
                 // DDLs and DMLs are considered not that frequent.
                 // limit the computation locations to the locations that will be used in the query
-                final INodeJobTracker nodeJobTracker = metadataProvider.getApplicationContext().getNodeJobTracker();
+                final INodeJobTracker nodeJobTracker = ccAppContext.getNodeJobTracker();
                 final AlgebricksAbsolutePartitionConstraint jobLocations =
                         getJobLocations(spec, nodeJobTracker, computationLocations);
                 final IClusterCapacity jobRequiredCapacity =
@@ -502,10 +483,10 @@
     }
 
     // Validates if the query contains unsupported query parameters.
-    private static Map<String, Object> validateConfig(Map<String, Object> config, SourceLocation sourceLoc)
+    private Map<String, Object> validateConfig(Map<String, Object> config, SourceLocation sourceLoc)
             throws AlgebricksException {
         for (String parameterName : config.keySet()) {
-            if (!CONFIGURABLE_PARAMETER_NAMES.contains(parameterName)
+            if (!configurableParameterNames.contains(parameterName)
                     && !parameterName.startsWith(PREFIX_INTERNAL_PARAMETERS)) {
                 throw AsterixException.create(ErrorCode.COMPILATION_UNSUPPORTED_QUERY_PARAMETER, sourceLoc,
                         parameterName);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
index 4dadf55..bf77c24 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
@@ -28,8 +28,6 @@
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.result.IResultSet;
 import org.apache.hyracks.http.server.AbstractServlet;
-import org.apache.hyracks.ipc.exceptions.IPCException;
-import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -61,13 +59,7 @@
     }
 
     protected IResultSet getResultSet() throws Exception { // NOSONAR
-        try {
-            return ServletUtil.getResultSet(getHyracksClientConnection(), appCtx, ctx);
-        } catch (IPCException e) {
-            LOGGER.log(Level.WARN, "Failed getting hyracks dataset connection. Resetting hyracks connection.", e);
-            ctx.put(HYRACKS_CONNECTION_ATTR, appCtx.getHcc());
-            return ServletUtil.getResultSet(getHyracksClientConnection(), appCtx, ctx);
-        }
+        return ServletUtil.getResultSet(appCtx, ctx);
     }
 
     protected IHyracksClientConnection getHyracksClientConnection() throws Exception { // NOSONAR
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
index 081c69a..56ad88e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
@@ -134,7 +134,7 @@
         try {
             // TODO: warnings should be retrieved from warnings collectors
             IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
-            IResultSet resultSet = ServletUtil.getResultSet(hcc, appCtx, ctx);
+            IResultSet resultSet = ServletUtil.getResultSet(appCtx, ctx);
             IParser parser = parserFactory.createParser(query);
             List<Statement> statements = parser.parse();
             SessionConfig sessionConfig = new SessionConfig(format, true, isSet(executeQuery), true, planFormat);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
index 981cdc9..bfebfd6 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
@@ -21,7 +21,6 @@
 import java.io.IOException;
 import java.util.concurrent.ConcurrentMap;
 
-import org.apache.asterix.api.common.ResultMetadata;
 import org.apache.asterix.app.result.ResponseMetrics;
 import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.app.result.ResultHandle;
@@ -31,6 +30,7 @@
 import org.apache.asterix.app.result.fields.ResultsPrinter;
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
+import org.apache.asterix.translator.ResultMetadata;
 import org.apache.asterix.translator.SessionConfig;
 import org.apache.asterix.translator.SessionOutput;
 import org.apache.hyracks.api.exceptions.ErrorCode;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
index 3ac37bf..8d5e4db 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
@@ -24,29 +24,22 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.commons.codec.DecoderException;
 import org.apache.commons.codec.net.URLCodec;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.result.IResultSet;
-import org.apache.hyracks.client.result.ResultSet;
 import org.apache.hyracks.http.api.IServletRequest;
 
 public class ServletUtil {
-    static IResultSet getResultSet(IHyracksClientConnection hcc, IApplicationContext appCtx,
-            final Map<String, Object> ctx) throws Exception {
+    static IResultSet getResultSet(IApplicationContext appCtx, final Map<String, Object> ctx) throws Exception {
         IResultSet resultSet = (IResultSet) ctx.get(RESULTSET_ATTR);
         if (resultSet == null) {
             synchronized (ctx) {
                 resultSet = (IResultSet) ctx.get(RESULTSET_ATTR);
                 if (resultSet == null) {
-                    resultSet = new ResultSet(hcc,
-                            appCtx.getServiceContext().getControllerService().getNetworkSecurityManager()
-                                    .getSocketChannelFactory(),
-                            appCtx.getCompilerProperties().getFrameSize(), ResultReader.NUM_READERS);
+                    resultSet = appCtx.getResultSet();
                     ctx.put(RESULTSET_ATTR, resultSet);
                 }
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CcApplicationContext.java
similarity index 87%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CcApplicationContext.java
index 66f0e73..880880e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CcApplicationContext.java
@@ -16,12 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.utils;
+package org.apache.asterix.app.cc;
 
 import java.io.IOException;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.Supplier;
 
+import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.common.api.IConfigValidator;
 import org.apache.asterix.common.api.IConfigValidatorFactory;
 import org.apache.asterix.common.api.ICoordinationService;
@@ -56,13 +57,21 @@
 import org.apache.asterix.runtime.compression.CompressionManager;
 import org.apache.asterix.runtime.job.listener.NodeJobTracker;
 import org.apache.asterix.runtime.transaction.ResourceIdManager;
+import org.apache.asterix.runtime.utils.BulkTxnIdFactory;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
+import org.apache.asterix.runtime.utils.NoOpCoordinationService;
+import org.apache.asterix.runtime.utils.RequestTracker;
+import org.apache.asterix.runtime.utils.RuntimeComponentsProvider;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.ICCServiceContext;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IJobLifecycleListener;
+import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.client.result.ResultSet;
 import org.apache.hyracks.ipc.impl.HyracksConnection;
 import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.util.NetworkUtil;
 
 /*
  * Acts as an holder class for IndexRegistryProvider, AsterixStorageManager
@@ -87,7 +96,8 @@
     private MessagingProperties messagingProperties;
     private NodeProperties nodeProperties;
     private Supplier<IMetadataBootstrap> metadataBootstrapSupplier;
-    private IHyracksClientConnection hcc;
+    private volatile HyracksConnection hcc;
+    private volatile ResultSet resultSet;
     private Object extensionManager;
     private INcLifecycleCoordinator ftStrategy;
     private IJobLifecycleListener activeLifeCycleListener;
@@ -103,7 +113,7 @@
     private final IAdapterFactoryService adapterFactoryService;
     private final ReentrantReadWriteLock compilationLock = new ReentrantReadWriteLock(true);
 
-    public CcApplicationContext(ICCServiceContext ccServiceCtx, IHyracksClientConnection hcc,
+    public CcApplicationContext(ICCServiceContext ccServiceCtx, HyracksConnection hcc,
             Supplier<IMetadataBootstrap> metadataBootstrapSupplier, IGlobalRecoveryManager globalRecoveryManager,
             INcLifecycleCoordinator ftStrategy, IJobLifecycleListener activeLifeCycleListener,
             IStorageComponentProvider storageComponentProvider, IMetadataLockManager mdLockManager,
@@ -188,18 +198,44 @@
 
     @Override
     public IHyracksClientConnection getHcc() throws HyracksDataException {
-        if (!hcc.isConnected()) {
+        HyracksConnection hc = hcc;
+        if (!hc.isConnected()) {
             synchronized (this) {
-                if (!hcc.isConnected()) {
+                hc = hcc;
+                if (!hc.isConnected()) {
                     try {
-                        hcc = new HyracksConnection(hcc.getHost(), hcc.getPort());
+                        ResultSet rs = resultSet;
+                        resultSet = null;
+                        NetworkUtil.closeQuietly(rs);
+
+                        NetworkUtil.closeQuietly(hc);
+                        hcc = hc = new HyracksConnection(hcc.getHost(), hcc.getPort());
                     } catch (Exception e) {
                         throw HyracksDataException.create(e);
                     }
                 }
             }
         }
-        return hcc;
+        return hc;
+    }
+
+    @Override
+    public IResultSet getResultSet() throws HyracksDataException {
+        ResultSet rs = resultSet;
+        if (rs == null) {
+            synchronized (this) {
+                rs = resultSet;
+                if (rs == null) {
+                    try {
+                        resultSet = rs = ResultReader.createResultSet(getHcc(), ccServiceCtx.getControllerService(),
+                                compilerProperties);
+                    } catch (Exception e) {
+                        throw HyracksDataException.create(e);
+                    }
+                }
+            }
+        }
+        return rs;
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
index 30eaf93..6c0382d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.ISecondaryIndexOperationsHelper;
 import org.apache.asterix.metadata.utils.SecondaryIndexOperationsHelper;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -63,7 +64,7 @@
         if (index.isPrimaryIndex()) {
             throw new CompilationException(ErrorCode.OPERATION_NOT_SUPPORTED_ON_PRIMARY_INDEX, loc, indexName);
         }
-        SecondaryIndexOperationsHelper secondaryIndexHelper =
+        ISecondaryIndexOperationsHelper secondaryIndexHelper =
                 SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, loc);
         IndexDataflowHelperFactory indexDataflowHelperFactory =
                 new IndexDataflowHelperFactory(metadataProvider.getStorageComponentProvider().getStorageManager(),
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index 1a89168..a46522e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -27,6 +27,7 @@
 import java.util.concurrent.ExecutorService;
 
 import org.apache.asterix.active.ActiveManager;
+import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.common.api.IConfigValidator;
 import org.apache.asterix.common.api.IConfigValidatorFactory;
 import org.apache.asterix.common.api.ICoordinationService;
@@ -86,6 +87,8 @@
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponentManager;
 import org.apache.hyracks.api.network.INetworkSecurityManager;
+import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.client.result.ResultSet;
 import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.control.nc.NodeControllerService;
 import org.apache.hyracks.ipc.impl.HyracksConnection;
@@ -108,6 +111,7 @@
 import org.apache.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
 import org.apache.hyracks.storage.common.file.IResourceIdFactory;
 import org.apache.hyracks.util.MaintainedThreadNameExecutorService;
+import org.apache.hyracks.util.NetworkUtil;
 import org.apache.hyracks.util.cache.CacheManager;
 import org.apache.hyracks.util.cache.ICacheManager;
 import org.apache.logging.log4j.Level;
@@ -147,7 +151,8 @@
     private final NCExtensionManager ncExtensionManager;
     private final IStorageComponentProvider componentProvider;
     private final IPersistedResourceRegistry persistedResourceRegistry;
-    private IHyracksClientConnection hcc;
+    private volatile HyracksConnection hcc;
+    private volatile ResultSet resultSet;
     private IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
     private IReplicaManager replicaManager;
     private IReceptionist receptionist;
@@ -513,15 +518,22 @@
 
     @Override
     public IHyracksClientConnection getHcc() throws HyracksDataException {
-        if (hcc == null || !hcc.isConnected()) {
+        HyracksConnection hc = hcc;
+        if (hc == null || !hc.isConnected()) {
             synchronized (this) {
-                if (hcc == null || !hcc.isConnected()) {
+                hc = hcc;
+                if (hc == null || !hc.isConnected()) {
                     try {
+                        ResultSet rs = resultSet;
+                        resultSet = null;
+                        NetworkUtil.closeQuietly(rs);
+
                         NodeControllerService ncSrv = (NodeControllerService) ncServiceContext.getControllerService();
                         // TODO(mblow): multicc
                         CcId primaryCcId = ncSrv.getPrimaryCcId();
                         ClusterControllerInfo ccInfo = ncSrv.getNodeParameters(primaryCcId).getClusterControllerInfo();
-                        hcc = new HyracksConnection(ccInfo.getClientNetAddress(), ccInfo.getClientNetPort(),
+                        NetworkUtil.closeQuietly(hc);
+                        hcc = hc = new HyracksConnection(ccInfo.getClientNetAddress(), ccInfo.getClientNetPort(),
                                 ncSrv.getNetworkSecurityManager().getSocketChannelFactory());
                     } catch (Exception e) {
                         throw HyracksDataException.create(e);
@@ -529,7 +541,26 @@
                 }
             }
         }
-        return hcc;
+        return hc;
+    }
+
+    @Override
+    public IResultSet getResultSet() throws HyracksDataException {
+        ResultSet rs = resultSet;
+        if (rs == null) {
+            synchronized (this) {
+                rs = resultSet;
+                if (rs == null) {
+                    try {
+                        resultSet = rs = ResultReader.createResultSet(getHcc(), ncServiceContext.getControllerService(),
+                                compilerProperties);
+                    } catch (Exception e) {
+                        throw HyracksDataException.create(e);
+                    }
+                }
+            }
+        }
+        return rs;
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/JobResultCallback.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/JobResultCallback.java
index 05073f9..201a470 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/JobResultCallback.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/JobResultCallback.java
@@ -23,8 +23,8 @@
 import java.util.Iterator;
 import java.util.Set;
 
-import org.apache.asterix.api.common.ResultMetadata;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.translator.ResultMetadata;
 import org.apache.hyracks.api.exceptions.Warning;
 import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.api.job.JobId;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
index 1acae87..56dfd5e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultReader.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.app.result;
 
+import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.comm.IFrame;
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -27,12 +29,14 @@
 import org.apache.hyracks.api.result.IResultSetReader;
 import org.apache.hyracks.api.result.ResultJobRecord.Status;
 import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.api.service.IControllerService;
+import org.apache.hyracks.client.result.ResultSet;
 import org.apache.hyracks.dataflow.common.comm.io.ResultFrameTupleAccessor;
 
 public class ResultReader {
-    private IResultSetReader reader;
+    private final IResultSetReader reader;
 
-    private IFrameTupleAccessor frameTupleAccessor;
+    private final IFrameTupleAccessor frameTupleAccessor;
 
     // Number of parallel result reader buffers
     public static final int NUM_READERS = 1;
@@ -57,4 +61,10 @@
     public IResultMetadata getMetadata() {
         return reader.getResultMetadata();
     }
+
+    public static ResultSet createResultSet(IHyracksClientConnection hcc, IControllerService srv,
+            CompilerProperties compilerProperties) throws Exception {
+        return new ResultSet(hcc, srv.getNetworkSecurityManager().getSocketChannelFactory(),
+                compilerProperties.getFrameSize(), ResultReader.NUM_READERS);
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
index 048584c..3f1e632 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
@@ -22,7 +22,6 @@
 import java.util.LinkedHashSet;
 import java.util.List;
 
-import org.apache.asterix.api.common.ResultMetadata;
 import org.apache.asterix.api.http.server.ResultUtil;
 import org.apache.asterix.common.annotations.IRecordTypeAnnotation;
 import org.apache.asterix.common.annotations.RecordFieldOrderAnnotation;
@@ -34,6 +33,7 @@
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.translator.ExecutionPlans;
+import org.apache.asterix.translator.ResultMetadata;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.util.JSONUtil;
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 41d9282..3b41beb 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -110,7 +110,9 @@
 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.AnalyzeStatement;
 import org.apache.asterix.lang.common.statement.CompactStatement;
 import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
@@ -195,6 +197,7 @@
 import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
 import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.asterix.metadata.utils.MetadataUtil;
+import org.apache.asterix.metadata.utils.SampleOperationsHelper;
 import org.apache.asterix.metadata.utils.TypeUtil;
 import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.base.IAObject;
@@ -208,6 +211,7 @@
 import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
 import org.apache.asterix.runtime.fulltext.IFullTextFilterDescriptor;
 import org.apache.asterix.runtime.fulltext.StopwordsFullTextFilterDescriptor;
+import org.apache.asterix.runtime.operators.StreamStats;
 import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
 import org.apache.asterix.translator.AbstractLangTranslator;
 import org.apache.asterix.translator.ClientRequest;
@@ -235,13 +239,11 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.core.algebra.base.Counter;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.data.IAWriterFactory;
-import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
-import org.apache.hyracks.algebricks.runtime.serializer.ResultSerializerFactoryProvider;
-import org.apache.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
 import org.apache.hyracks.api.client.IClusterInfoCollector;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -253,6 +255,7 @@
 import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 import org.apache.hyracks.api.result.IResultSet;
 import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
@@ -330,10 +333,8 @@
     public void compileAndExecute(IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
         validateStatements(requestParameters);
         trackRequest(requestParameters);
-        int resultSetIdCounter = 0;
+        Counter resultSetIdCounter = new Counter(0);
         FileSplit outputFile = null;
-        IAWriterFactory writerFactory = PrinterBasedWriterFactory.INSTANCE;
-        IResultSerializerFactoryProvider resultSerializerFactoryProvider = ResultSerializerFactoryProvider.INSTANCE;
         String threadName = Thread.currentThread().getName();
         Thread.currentThread().setName(
                 QueryTranslator.class.getSimpleName() + ":" + requestParameters.getRequestReference().getUuid());
@@ -353,8 +354,8 @@
                 }
                 validateOperation(appCtx, activeDataverse, stmt);
                 MetadataProvider metadataProvider = MetadataProvider.create(appCtx, activeDataverse);
-                configureMetadataProvider(metadataProvider, config, resultSerializerFactoryProvider, writerFactory,
-                        outputFile, requestParameters, stmt);
+                configureMetadataProvider(metadataProvider, config, resultSetIdCounter, outputFile, requestParameters,
+                        stmt);
                 IStatementRewriter stmtRewriter = rewriterFactory.createStatementRewriter();
                 rewriteStatement(stmt, stmtRewriter, metadataProvider); // Rewrite the statement's AST.
                 Statement.Kind kind = stmt.getKind();
@@ -444,7 +445,7 @@
                     case INSERT:
                     case UPSERT:
                         if (((InsertStatement) stmt).getReturnExpression() != null) {
-                            metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
+                            metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter.getAndInc()));
                             metadataProvider.setResultAsyncMode(resultDelivery == ResultDelivery.ASYNC
                                     || resultDelivery == ResultDelivery.DEFERRED);
                             metadataProvider.setMaxResultReads(maxResultReads);
@@ -480,7 +481,7 @@
                         handleCreateFeedPolicyStatement(metadataProvider, stmt);
                         break;
                     case QUERY:
-                        metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
+                        metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter.getAndInc()));
                         metadataProvider.setResultAsyncMode(
                                 resultDelivery == ResultDelivery.ASYNC || resultDelivery == ResultDelivery.DEFERRED);
                         metadataProvider.setMaxResultReads(maxResultReads);
@@ -490,6 +491,9 @@
                         handleQuery(metadataProvider, (Query) stmt, hcc, resultSet, resultDelivery, outMetadata, stats,
                                 requestParameters, stmtParams, stmtRewriter);
                         break;
+                    case ANALYZE:
+                        handleAnalyzeStatement(metadataProvider, stmt, hcc, requestParameters);
+                        break;
                     case COMPACT:
                         handleCompactStatement(metadataProvider, stmt, hcc);
                         break;
@@ -505,7 +509,10 @@
                     case EXTENSION:
                         final ExtensionStatement extStmt = (ExtensionStatement) stmt;
                         statementProperties.setName(extStmt.getName());
-                        extStmt.handle(hcc, this, requestParameters, metadataProvider, resultSetIdCounter);
+                        if (!isCompileOnly()) {
+                            extStmt.handle(hcc, this, requestParameters, metadataProvider,
+                                    resultSetIdCounter.getAndInc());
+                        }
                         break;
                     default:
                         throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, stmt.getSourceLocation(),
@@ -522,14 +529,13 @@
     }
 
     protected void configureMetadataProvider(MetadataProvider metadataProvider, Map<String, String> config,
-            IResultSerializerFactoryProvider resultSerializerFactoryProvider, IAWriterFactory writerFactory,
-            FileSplit outputFile, IRequestParameters requestParameters, Statement statement) {
+            Counter resultSetIdCounter, FileSplit outputFile, IRequestParameters requestParameters,
+            Statement statement) {
         if (statement.getKind() == Statement.Kind.QUERY && requestParameters.isSQLCompatMode()) {
             metadataProvider.getConfig().put(SqlppQueryRewriter.SQL_COMPAT_OPTION, Boolean.TRUE.toString());
         }
         metadataProvider.getConfig().putAll(config);
-        metadataProvider.setWriterFactory(writerFactory);
-        metadataProvider.setResultSerializerFactoryProvider(resultSerializerFactoryProvider);
+        metadataProvider.setResultSetIdCounter(resultSetIdCounter);
         metadataProvider.setOutputFile(outputFile);
     }
 
@@ -552,6 +558,13 @@
         config.put(pname, pvalue);
     }
 
+    protected LangRewritingContext createLangRewritingContext(MetadataProvider metadataProvider,
+            List<FunctionDecl> declaredFunctions, List<ViewDecl> declaredViews, IWarningCollector warningCollector,
+            int varCounter) {
+        return new LangRewritingContext(metadataProvider, declaredFunctions, declaredViews, warningCollector,
+                varCounter);
+    }
+
     protected Pair<IAWriterFactory, FileSplit> handleWriteStatement(Statement stmt)
             throws InstantiationException, IllegalAccessException, ClassNotFoundException {
         WriteStatement ws = (WriteStatement) stmt;
@@ -610,6 +623,9 @@
         CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
         DataverseName dvName = stmtCreateDataverse.getDataverseName();
         metadataProvider.validateDataverseName(dvName, stmtCreateDataverse.getSourceLocation());
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createDataverseBegin(lockManager, metadataProvider.getLocks(), dvName);
         try {
             doCreateDataverseStatement(metadataProvider, stmtCreateDataverse, requestParameters);
@@ -713,6 +729,9 @@
         String compactionPolicy = dd.getCompactionPolicy();
         boolean defaultCompactionPolicy = compactionPolicy == null;
 
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName,
                 itemTypeDataverseName, itemTypeName, itemTypeAnonymous, metaItemTypeDataverseName, metaItemTypeName,
                 metaItemTypeAnonymous, nodegroupName, compactionPolicy, defaultCompactionPolicy, dd.getDatasetType(),
@@ -1061,6 +1080,9 @@
         metadataProvider.validateDatabaseObjectName(stmtCreateIndex.getDataverseName(), indexName,
                 stmt.getSourceLocation());
         DataverseName dataverseName = getActiveDataverseName(stmtCreateIndex.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName,
                 fullTextConfigName);
         try {
@@ -1409,6 +1431,9 @@
                 stmt.getSourceLocation());
         DataverseName dataverseName = getActiveDataverseName(stmtCreateFilter.getDataverseName());
 
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createFullTextFilterBegin(lockManager, metadataProvider.getLocks(), dataverseName, fullTextFilterName);
         try {
             doCreateFullTextFilter(metadataProvider, stmtCreateFilter, dataverseName);
@@ -1477,6 +1502,9 @@
         DataverseName dataverseName = getActiveDataverseName(stmtCreateConfig.getDataverseName());
         ImmutableList<String> filterNames = stmtCreateConfig.getFilterNames();
 
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createFullTextConfigBegin(lockManager, metadataProvider.getLocks(), dataverseName, configName,
                 filterNames);
         try {
@@ -1842,6 +1870,9 @@
         String typeName = stmtCreateType.getIdent().getValue();
         metadataProvider.validateDatabaseObjectName(stmtCreateType.getDataverseName(), typeName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(stmtCreateType.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockUtil.createTypeBegin(lockManager, metadataProvider.getLocks(), dataverseName, typeName);
@@ -1895,6 +1926,9 @@
             throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                     dataverseName + " " + dataverse() + " can't be dropped");
         }
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropDataverseBegin(lockManager, metadataProvider.getLocks(), dataverseName);
         try {
             doDropDataverse(stmtDropDataverse, metadataProvider, hcc, requestParameters);
@@ -2091,6 +2125,9 @@
         String datasetName = stmtDelete.getDatasetName().getValue();
         metadataProvider.validateDatabaseObjectName(stmtDelete.getDataverseName(), datasetName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(stmtDelete.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
         try {
             doDropDataset(dataverseName, datasetName, metadataProvider, stmtDelete.getIfExists(), hcc,
@@ -2198,6 +2235,9 @@
                 stmtIndexDrop.getIndexName().getValue(), stmtIndexDrop.getSourceLocation());
         DataverseName dataverseName = getActiveDataverseName(stmtIndexDrop.getDataverseName());
         String datasetName = stmtIndexDrop.getDatasetName().getValue();
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
         try {
             doDropIndex(metadataProvider, stmtIndexDrop, dataverseName, datasetName, hcc, requestParameters);
@@ -2380,6 +2420,9 @@
         DataverseName dataverseName = getActiveDataverseName(stmtFilterDrop.getDataverseName());
         String fullTextFilterName = stmtFilterDrop.getFilterName();
 
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropFullTextFilterBegin(lockManager, metadataProvider.getLocks(), dataverseName, fullTextFilterName);
         try {
             doDropFullTextFilter(metadataProvider, stmtFilterDrop, dataverseName, fullTextFilterName);
@@ -2420,6 +2463,9 @@
         DataverseName dataverseName = getActiveDataverseName(stmtConfigDrop.getDataverseName());
         String configName = stmtConfigDrop.getConfigName();
 
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropFullTextConfigBegin(lockManager, metadataProvider.getLocks(), dataverseName, configName);
         try {
             doDropFullTextConfig(metadataProvider, stmtConfigDrop, hcc, requestParameters);
@@ -2469,6 +2515,9 @@
         String typeName = stmtTypeDrop.getTypeName().getValue();
         metadataProvider.validateDatabaseObjectName(stmtTypeDrop.getDataverseName(), typeName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(stmtTypeDrop.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockUtil.dropTypeBegin(lockManager, metadataProvider.getLocks(), dataverseName, typeName);
@@ -2509,6 +2558,9 @@
         SourceLocation sourceLoc = stmtDelete.getSourceLocation();
         String nodegroupName = stmtDelete.getNodeGroupName().getValue();
         metadataProvider.validateDatabaseObjectName(null, nodegroupName, sourceLoc);
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockManager.acquireNodeGroupWriteLock(metadataProvider.getLocks(), nodegroupName);
@@ -2553,6 +2605,9 @@
             viewItemTypeAnonymous = false;
         }
 
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, viewName,
                 viewItemTypeDataverseName, viewItemTypeName, viewItemTypeAnonymous, null, null, false, null, null, true,
                 DatasetType.VIEW, null);
@@ -2707,8 +2762,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);
@@ -2744,6 +2801,9 @@
         String viewName = stmtDrop.getViewName().getValue();
         metadataProvider.validateDatabaseObjectName(stmtDrop.getDataverseName(), viewName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(stmtDrop.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, viewName);
         try {
             doDropView(metadataProvider, stmtDrop, dataverseName, viewName);
@@ -2826,6 +2886,9 @@
             }
         }
 
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createFunctionBegin(lockManager, metadataProvider.getLocks(), dataverseName, signature.getName(),
                 libraryDataverseName, libraryName);
         try {
@@ -2976,8 +3039,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);
@@ -3075,6 +3140,9 @@
                 stmtDropFunction.getSourceLocation());
         DataverseName dataverseName = getActiveDataverseName(signature.getDataverseName());
         signature.setDataverseName(dataverseName);
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropFunctionBegin(lockManager, metadataProvider.getLocks(), dataverseName, signature.getName());
         try {
             doDropFunction(metadataProvider, stmtDropFunction, signature, requestParameters);
@@ -3133,6 +3201,9 @@
             libraryDataverseName = dataverseName;
         }
         String libraryName = cas.getLibraryName();
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createAdapterBegin(lockManager, metadataProvider.getLocks(), dataverseName, adapterName,
                 libraryDataverseName, libraryName);
         try {
@@ -3203,6 +3274,9 @@
         String adapterName = stmtDropAdapter.getAdapterName();
         metadataProvider.validateDatabaseObjectName(stmtDropAdapter.getDataverseName(), adapterName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(stmtDropAdapter.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropAdapterBegin(lockManager, metadataProvider.getLocks(), dataverseName, adapterName);
         try {
             doDropAdapter(metadataProvider, stmtDropAdapter, dataverseName, adapterName);
@@ -3253,6 +3327,9 @@
         DataverseName dataverseName = getActiveDataverseName(cls.getDataverseName());
         String libraryName = cls.getLibraryName();
         String libraryHash = cls.getHash();
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createLibraryBegin(lockManager, metadataProvider.getLocks(), dataverseName, libraryName);
         try {
             doCreateLibrary(metadataProvider, dataverseName, libraryName, libraryHash, cls, hcc, requestParameters);
@@ -3381,6 +3458,9 @@
         metadataProvider.validateDatabaseObjectName(stmtDropLibrary.getDataverseName(), libraryName,
                 stmtDropLibrary.getSourceLocation());
         DataverseName dataverseName = getActiveDataverseName(stmtDropLibrary.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropLibraryBegin(lockManager, metadataProvider.getLocks(), dataverseName, libraryName);
         try {
             doDropLibrary(metadataProvider, stmtDropLibrary, dataverseName, libraryName, hcc, requestParameters);
@@ -3475,6 +3555,9 @@
         DataverseName objectDataverseName =
                 css.getObjectDataverseName() != null ? css.getObjectDataverseName() : dataverseName;
         String objectName = css.getObjectName();
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createSynonymBegin(lockManager, metadataProvider.getLocks(), dataverseName, synonymName);
         try {
             doCreateSynonym(metadataProvider, css, dataverseName, synonymName, objectDataverseName, objectName);
@@ -3522,6 +3605,9 @@
         metadataProvider.validateDatabaseObjectName(stmtSynDrop.getDataverseName(), synonymName,
                 stmtSynDrop.getSourceLocation());
         DataverseName dataverseName = getActiveDataverseName(stmtSynDrop.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.dropSynonymBegin(lockManager, metadataProvider.getLocks(), dataverseName, synonymName);
         try {
             doDropSynonym(metadataProvider, stmtSynDrop, dataverseName, synonymName);
@@ -3575,7 +3661,7 @@
             afterCompile();
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
-            if (spec != null && sessionConfig.isExecuteQuery()) {
+            if (spec != null && !isCompileOnly()) {
                 runJob(hcc, spec);
             }
         } catch (Exception e) {
@@ -3620,7 +3706,7 @@
                         rewriteCompileInsertUpsert(hcc, metadataProvider, stmtInsertUpsert, stmtParams);
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                 bActiveTxn = false;
-                return !sessionConfig.isExecuteQuery() ? null : jobSpec;
+                return isCompileOnly() ? null : jobSpec;
             } catch (Exception e) {
                 if (bActiveTxn) {
                     abort(e, e, mdTxnCtx);
@@ -3671,7 +3757,7 @@
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
 
-            if (jobSpec != null && sessionConfig.isExecuteQuery()) {
+            if (jobSpec != null && !isCompileOnly()) {
                 runJob(hcc, jobSpec);
             }
             return jobSpec;
@@ -3694,8 +3780,10 @@
         Map<VarIdentifier, IAObject> externalVars = createExternalVariables(query, stmtParams);
 
         // Query Rewriting (happens under the same ongoing metadata transaction)
-        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions, null,
-                metadataProvider, query, sessionOutput, true, true, externalVars.keySet(), warningCollector);
+        LangRewritingContext langRewritingContext = createLangRewritingContext(metadataProvider, declaredFunctions,
+                null, warningCollector, query.getVarCounter());
+        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(langRewritingContext, query,
+                sessionOutput, true, true, externalVars.keySet());
 
         // Query Compilation (happens under the same ongoing metadata transaction)
         return apiFramework.compileQuery(clusterInfoCollector, metadataProvider, (Query) rewrittenResult.first,
@@ -3703,7 +3791,7 @@
                 responsePrinter, warningCollector, requestParameters);
     }
 
-    private JobSpecification rewriteCompileInsertUpsert(IClusterInfoCollector clusterInfoCollector,
+    protected JobSpecification rewriteCompileInsertUpsert(IClusterInfoCollector clusterInfoCollector,
             MetadataProvider metadataProvider, InsertStatement insertUpsert, Map<String, IAObject> stmtParams)
             throws AlgebricksException, ACIDException {
         SourceLocation sourceLoc = insertUpsert.getSourceLocation();
@@ -3711,8 +3799,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());
@@ -3748,6 +3838,9 @@
         String feedName = cfs.getFeedName().getValue();
         metadataProvider.validateDatabaseObjectName(cfs.getDataverseName(), feedName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(cfs.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockUtil.createFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, feedName);
@@ -3787,6 +3880,9 @@
         String policyName = cfps.getPolicyName();
         metadataProvider.validateDatabaseObjectName(null, policyName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(null);
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.createFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverseName, policyName);
         try {
             mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -3848,6 +3944,9 @@
         String feedName = stmtFeedDrop.getFeedName().getValue();
         metadataProvider.validateDatabaseObjectName(stmtFeedDrop.getDataverseName(), feedName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(stmtFeedDrop.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockUtil.dropFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, feedName);
@@ -3901,6 +4000,9 @@
         String policyName = stmtFeedPolicyDrop.getPolicyName().getValue();
         metadataProvider.validateDatabaseObjectName(stmtFeedPolicyDrop.getDataverseName(), policyName, sourceLoc);
         DataverseName dataverseName = getActiveDataverseName(stmtFeedPolicyDrop.getDataverseName());
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockUtil.dropFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverseName, policyName);
@@ -3930,6 +4032,9 @@
         SourceLocation sourceLoc = sfs.getSourceLocation();
         DataverseName dataverseName = getActiveDataverseName(sfs.getDataverseName());
         String feedName = sfs.getFeedName().getValue();
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean committed = false;
         lockUtil.startFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, feedName);
@@ -3992,6 +4097,9 @@
             throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                     "Feed " + feedName + " is not started.");
         }
+        if (isCompileOnly()) {
+            return;
+        }
         lockUtil.stopFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, feedName);
         try {
             listener.stop(metadataProvider);
@@ -4009,6 +4117,9 @@
         String datasetName = cfs.getDatasetName().getValue();
         String policyName = cfs.getPolicy();
         String whereClauseBody = cfs.getWhereClauseBody();
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         // TODO: Check whether we are connecting a change feed to a non-meta dataset
@@ -4064,6 +4175,9 @@
         DataverseName dataverseName = getActiveDataverseName(cfs.getDataverseName());
         String datasetName = cfs.getDatasetName().getValue();
         String feedName = cfs.getFeedName().getValue();
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockUtil.disconnectFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName, feedName);
@@ -4103,12 +4217,236 @@
         }
     }
 
+    protected void handleAnalyzeStatement(MetadataProvider metadataProvider, Statement stmt,
+            IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
+        AnalyzeStatement analyzeStatement = (AnalyzeStatement) stmt;
+        metadataProvider.validateDatabaseObjectName(analyzeStatement.getDataverseName(),
+                analyzeStatement.getDatasetName(), analyzeStatement.getSourceLocation());
+        DataverseName dataverseName = getActiveDataverseName(analyzeStatement.getDataverseName());
+        String datasetName = analyzeStatement.getDatasetName();
+        if (isCompileOnly()) {
+            return;
+        }
+        lockUtil.analyzeDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
+        try {
+            doAnalyzeDataset(metadataProvider, analyzeStatement, dataverseName, datasetName, hcc, requestParameters);
+        } finally {
+            metadataProvider.getLocks().unlock();
+        }
+    }
+
+    protected void doAnalyzeDataset(MetadataProvider metadataProvider, AnalyzeStatement stmtAnalyze,
+            DataverseName dataverseName, String datasetName, IHyracksClientConnection hcc,
+            IRequestParameters requestParameters) throws Exception {
+        SourceLocation sourceLoc = stmtAnalyze.getSourceLocation();
+        ProgressState progressNewIndexCreate = ProgressState.NO_PROGRESS;
+        ProgressState progressExistingIndexDrop = ProgressState.NO_PROGRESS;
+        Dataset ds = null;
+        Index existingIndex = null, newIndexPendingAdd = null;
+        JobSpecification existingIndexDropSpec = null;
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        boolean bActiveTxn = true;
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        try {
+            // Check if the dataverse exists
+            Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
+            if (dv == null) {
+                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
+            }
+            // Check if the dataset exists
+            ds = metadataProvider.findDataset(dataverseName, datasetName);
+            if (ds == null) {
+                throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+                        dataverseName);
+            }
+            if (ds.getDatasetType() == DatasetType.INTERNAL) {
+                validateDatasetState(metadataProvider, ds, sourceLoc);
+            } else {
+                throw new CompilationException(ErrorCode.OPERATION_NOT_SUPPORTED, sourceLoc);
+            }
+
+            IndexType sampleIndexType = IndexType.SAMPLE;
+            Pair<String, String> sampleIndexNames = IndexUtil.getSampleIndexNames(datasetName);
+            String newIndexName;
+            existingIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+                    datasetName, sampleIndexNames.first);
+            if (existingIndex != null) {
+                newIndexName = sampleIndexNames.second;
+            } else {
+                existingIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
+                        dataverseName, datasetName, sampleIndexNames.second);
+                newIndexName = sampleIndexNames.first;
+            }
+
+            InternalDatasetDetails dsDetails = (InternalDatasetDetails) ds.getDatasetDetails();
+            int sampleCardinalityTarget = stmtAnalyze.getSampleSize();
+            long sampleSeed = stmtAnalyze.getOrCreateSampleSeed();
+
+            Index.SampleIndexDetails newIndexDetailsPendingAdd =
+                    new Index.SampleIndexDetails(dsDetails.getPrimaryKey(), dsDetails.getKeySourceIndicator(),
+                            dsDetails.getPrimaryKeyType(), sampleCardinalityTarget, 0, 0, sampleSeed);
+            newIndexPendingAdd = new Index(dataverseName, datasetName, newIndexName, sampleIndexType,
+                    newIndexDetailsPendingAdd, false, false, MetadataUtil.PENDING_ADD_OP);
+
+            // #. add a new index with PendingAddOp
+            MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), newIndexPendingAdd);
+            // #. prepare to create the index artifact in NC.
+            JobSpecification spec =
+                    IndexUtil.buildSecondaryIndexCreationJobSpec(ds, newIndexPendingAdd, metadataProvider, sourceLoc);
+            if (spec == null) {
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Failed to create job spec for creating index '" + ds.getDatasetName() + "."
+                                + newIndexPendingAdd.getIndexName() + "'");
+            }
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
+            progressNewIndexCreate = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
+
+            // #. create the index artifact in NC.
+            runJob(hcc, spec);
+
+            // #. flush dataset
+            FlushDatasetUtil.flushDataset(hcc, metadataProvider, dataverseName, datasetName);
+
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            bActiveTxn = true;
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
+
+            // #. load data into the index in NC.
+            spec = IndexUtil.buildSecondaryIndexLoadingJobSpec(ds, newIndexPendingAdd, metadataProvider, sourceLoc);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
+
+            List<IOperatorStats> opStats = runJob(hcc, spec, jobFlags,
+                    Collections.singletonList(SampleOperationsHelper.DATASET_STATS_OPERATOR_NAME));
+            if (opStats == null || opStats.size() == 0) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, "", sourceLoc);
+            }
+            StreamStats stats = new StreamStats(opStats.get(0));
+
+            Index.SampleIndexDetails newIndexDetailsFinal = new Index.SampleIndexDetails(dsDetails.getPrimaryKey(),
+                    dsDetails.getKeySourceIndicator(), dsDetails.getPrimaryKeyType(), sampleCardinalityTarget,
+                    stats.getCardinality(), stats.getAvgTupleSize(), sampleSeed);
+            Index newIndexFinal = new Index(dataverseName, datasetName, newIndexName, sampleIndexType,
+                    newIndexDetailsFinal, false, false, MetadataUtil.PENDING_NO_OP);
+
+            // #. begin new metadataTxn
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            bActiveTxn = true;
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
+            // #. add same new index with PendingNoOp after deleting its entry with PendingAddOp
+            MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(),
+                    newIndexPendingAdd.getDataverseName(), newIndexPendingAdd.getDatasetName(),
+                    newIndexPendingAdd.getIndexName());
+            MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), newIndexFinal);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
+            progressNewIndexCreate = ProgressState.NO_PROGRESS;
+
+            if (existingIndex != null) {
+                // #. set existing index to PendingDropOp because we'll be dropping it next
+                mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+                bActiveTxn = true;
+                metadataProvider.setMetadataTxnContext(mdTxnCtx);
+                MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(),
+                        existingIndex.getDataverseName(), existingIndex.getDatasetName(), existingIndex.getIndexName());
+                existingIndex.setPendingOp(MetadataUtil.PENDING_DROP_OP);
+                MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), existingIndex);
+                existingIndexDropSpec = IndexUtil.buildDropIndexJobSpec(existingIndex, metadataProvider, ds, sourceLoc);
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                progressExistingIndexDrop = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
+                bActiveTxn = false;
+
+                // #. drop existing index on NCs
+                runJob(hcc, existingIndexDropSpec);
+
+                // #. drop existing index metadata
+                mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+                bActiveTxn = true;
+                metadataProvider.setMetadataTxnContext(mdTxnCtx);
+                MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(),
+                        existingIndex.getDataverseName(), existingIndex.getDatasetName(), existingIndex.getIndexName());
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                bActiveTxn = false;
+                progressExistingIndexDrop = ProgressState.NO_PROGRESS;
+            }
+
+        } catch (Exception e) {
+            LOGGER.error("failed to analyze dataset; executing compensating operations", e);
+            if (bActiveTxn) {
+                abort(e, e, mdTxnCtx);
+            }
+
+            if (progressExistingIndexDrop == ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA) {
+                // #. execute compensation operations remove the index in NCs
+                try {
+                    runJob(hcc, existingIndexDropSpec);
+                } catch (Exception e2) {
+                    // do no throw exception since still the metadata needs to be compensated.
+                    e.addSuppressed(e2);
+                }
+                // #. remove the record from the metadata.
+                mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+                metadataProvider.setMetadataTxnContext(mdTxnCtx);
+                try {
+                    MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(),
+                            existingIndex.getDataverseName(), existingIndex.getDatasetName(),
+                            existingIndex.getIndexName());
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                } catch (Exception e2) {
+                    e.addSuppressed(e2);
+                    abort(e, e2, mdTxnCtx);
+                    throw new IllegalStateException("System is inconsistent state: pending index("
+                            + existingIndex.getDataverseName() + "." + existingIndex.getDatasetName() + "."
+                            + existingIndex.getIndexName() + ") couldn't be removed from the metadata", e);
+                }
+            } else if (progressNewIndexCreate == ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA) {
+                // #. execute compensation operations remove the index in NCs
+                mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+                bActiveTxn = true;
+                metadataProvider.setMetadataTxnContext(mdTxnCtx);
+                try {
+                    JobSpecification jobSpec =
+                            IndexUtil.buildDropIndexJobSpec(newIndexPendingAdd, metadataProvider, ds, sourceLoc);
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                    bActiveTxn = false;
+                    runJob(hcc, jobSpec);
+                } catch (Exception e2) {
+                    e.addSuppressed(e2);
+                    if (bActiveTxn) {
+                        abort(e, e2, mdTxnCtx);
+                    }
+                }
+                // #. remove the record from the metadata.
+                mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+                metadataProvider.setMetadataTxnContext(mdTxnCtx);
+                try {
+                    MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(),
+                            newIndexPendingAdd.getDataverseName(), newIndexPendingAdd.getDatasetName(),
+                            newIndexPendingAdd.getIndexName());
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                } catch (Exception e2) {
+                    e.addSuppressed(e2);
+                    abort(e, e2, mdTxnCtx);
+                    throw new IllegalStateException("System is in inconsistent state: pending index("
+                            + newIndexPendingAdd.getDataverseName() + "." + newIndexPendingAdd.getDatasetName() + "."
+                            + newIndexPendingAdd.getIndexName() + ") couldn't be removed from the metadata", e);
+                }
+            }
+
+            throw e;
+        }
+    }
+
     protected void handleCompactStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc) throws Exception {
         CompactStatement compactStatement = (CompactStatement) stmt;
         SourceLocation sourceLoc = compactStatement.getSourceLocation();
         DataverseName dataverseName = getActiveDataverseName(compactStatement.getDataverseName());
         String datasetName = compactStatement.getDatasetName().getValue();
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -4212,7 +4550,7 @@
                 afterCompile();
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                 bActiveTxn = false;
-                return query.isExplain() || !sessionConfig.isExecuteQuery() ? null : jobSpec;
+                return query.isExplain() || isCompileOnly() ? null : jobSpec;
             } catch (Exception e) {
                 LOGGER.log(Level.INFO, e.getMessage(), e);
                 if (bActiveTxn) {
@@ -4270,8 +4608,8 @@
     private void updateJobStats(JobId jobId, Stats stats, ResultSetId rsId) throws HyracksDataException {
         final ClusterControllerService controllerService =
                 (ClusterControllerService) appCtx.getServiceContext().getControllerService();
-        org.apache.asterix.api.common.ResultMetadata resultMetadata =
-                (org.apache.asterix.api.common.ResultMetadata) controllerService.getResultDirectoryService()
+        org.apache.asterix.translator.ResultMetadata resultMetadata =
+                (org.apache.asterix.translator.ResultMetadata) controllerService.getResultDirectoryService()
                         .getResultMetadata(jobId, rsId);
         stats.setProcessedObjects(resultMetadata.getProcessedObjects());
         if (jobFlags.contains(JobFlag.PROFILE_RUNTIME)) {
@@ -4329,6 +4667,12 @@
         JobUtils.runJob(hcc, jobSpec, jobFlags, true);
     }
 
+    private static List<IOperatorStats> runJob(IHyracksClientConnection hcc, JobSpecification jobSpec,
+            EnumSet<JobFlag> jobFlags, List<String> statOperatorNames) throws Exception {
+        Pair<JobId, List<IOperatorStats>> p = JobUtils.runJob(hcc, jobSpec, jobFlags, true, statOperatorNames);
+        return p.second;
+    }
+
     private static void createAndRunJob(IHyracksClientConnection hcc, EnumSet<JobFlag> jobFlags, Mutable<JobId> jId,
             IStatementCompiler compiler, IMetadataLocker locker, ResultDelivery resultDelivery, IResultPrinter printer,
             IRequestParameters requestParameters, boolean cancellable, ICcApplicationContext appCtx,
@@ -4382,6 +4726,9 @@
         String ngName = stmtCreateNodegroup.getNodegroupName().getValue();
         metadataProvider.validateDatabaseObjectName(null, ngName, sourceLoc);
 
+        if (isCompileOnly()) {
+            return;
+        }
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockManager.acquireNodeGroupWriteLock(metadataProvider.getLocks(), ngName);
@@ -4416,9 +4763,6 @@
         DataverseName dataverseName = getActiveDataverseName(stmtRefresh.getDataverseName());
         String datasetName = stmtRefresh.getDatasetName().getValue();
         TransactionState transactionState = TransactionState.COMMIT;
-        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-        boolean bActiveTxn = true;
-        metadataProvider.setMetadataTxnContext(mdTxnCtx);
         JobSpecification spec = null;
         Dataset ds = null;
         List<ExternalFile> metadataFiles = null;
@@ -4429,6 +4773,12 @@
         Dataset transactionDataset = null;
         boolean lockAquired = false;
         boolean success = false;
+        if (isCompileOnly()) {
+            return;
+        }
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        boolean bActiveTxn = true;
         lockUtil.refreshDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
         try {
             ds = metadataProvider.findDataset(dataverseName, datasetName);
@@ -4763,7 +5113,7 @@
 
     private Map<VarIdentifier, IAObject> createExternalVariables(IReturningStatement stmt,
             Map<String, IAObject> stmtParams) throws CompilationException {
-        if (sessionConfig.isExecuteQuery()) {
+        if (!isCompileOnly()) {
             if (stmtParams == null || stmtParams.isEmpty()) {
                 return Collections.emptyMap();
             }
@@ -4788,6 +5138,10 @@
         }
     }
 
+    protected boolean isCompileOnly() {
+        return !sessionConfig.isExecuteQuery();
+    }
+
     protected void validateDatasetState(MetadataProvider metadataProvider, Dataset dataset, SourceLocation sourceLoc)
             throws Exception {
         validateIfResourceIsActiveInFeed(metadataProvider.getApplicationContext(), dataset, sourceLoc);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index 8f3deb1..2a66cfd 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -53,6 +53,7 @@
 import org.apache.asterix.api.http.server.VersionApiServlet;
 import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.app.cc.CCExtensionManager;
+import org.apache.asterix.app.cc.CcApplicationContext;
 import org.apache.asterix.app.config.ConfigValidator;
 import org.apache.asterix.app.io.PersistedResourceRegistry;
 import org.apache.asterix.app.replication.NcLifecycleCoordinator;
@@ -85,7 +86,6 @@
 import org.apache.asterix.metadata.lock.MetadataLockManager;
 import org.apache.asterix.metadata.utils.MetadataLockUtil;
 import org.apache.asterix.runtime.job.resource.JobCapacityController;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.asterix.translator.IStatementExecutorFactory;
 import org.apache.asterix.translator.Receptionist;
 import org.apache.asterix.util.MetadataBuiltinFunctions;
@@ -126,7 +126,7 @@
     protected WebManager webManager;
     protected ICcApplicationContext appCtx;
     private IJobCapacityController jobCapacityController;
-    private IHyracksClientConnection hcc;
+    private HyracksConnection hcc;
 
     @Override
     public void init(IServiceContext serviceCtx) throws Exception {
@@ -209,7 +209,7 @@
             IReceptionistFactory receptionistFactory, IConfigValidatorFactory configValidatorFactory,
             CCExtensionManager ccExtensionManager, IAdapterFactoryService adapterFactoryService)
             throws AlgebricksException, IOException {
-        return new CcApplicationContext(ccServiceCtx, getHcc(), () -> MetadataManager.INSTANCE, globalRecoveryManager,
+        return new CcApplicationContext(ccServiceCtx, hcc, () -> MetadataManager.INSTANCE, globalRecoveryManager,
                 lifecycleCoordinator, new ActiveNotificationHandler(), componentProvider, new MetadataLockManager(),
                 createMetadataLockUtil(), receptionistFactory, configValidatorFactory, ccExtensionManager,
                 adapterFactoryService);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
index 156b78a..48cf511 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
@@ -20,7 +20,7 @@
 
 import static java.util.regex.Pattern.CASE_INSENSITIVE;
 import static java.util.regex.Pattern.DOTALL;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.SECRET_ACCESS_KEY_FIELD_NAME;
 
 import java.util.regex.Pattern;
 
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiServletTest.java
index 8dcfa26..eb63218 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiServletTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionApiServletTest.java
@@ -31,8 +31,8 @@
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.asterix.api.http.server.VersionApiServlet;
+import org.apache.asterix.app.cc.CcApplicationContext;
 import org.apache.asterix.common.config.BuildProperties;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index 0b80881..34696b1 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -26,6 +26,7 @@
 import java.util.List;
 import java.util.Map;
 
+import org.apache.asterix.app.cc.CcApplicationContext;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.app.nc.TransactionSubsystem;
 import org.apache.asterix.common.config.TransactionProperties;
@@ -61,7 +62,6 @@
 import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorNodePushable;
 import org.apache.asterix.runtime.operators.LSMPrimaryInsertOperatorNodePushable;
 import org.apache.asterix.runtime.operators.LSMPrimaryUpsertOperatorNodePushable;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.asterix.test.runtime.ExecutionTestUtil;
 import org.apache.asterix.transaction.management.runtime.CommitRuntime;
 import org.apache.asterix.transaction.management.service.logging.LogReader;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
index b80fa30..a2a3b48 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
@@ -26,6 +26,7 @@
 import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.asterix.app.cc.CcApplicationContext;
 import org.apache.asterix.app.replication.NcLifecycleCoordinator;
 import org.apache.asterix.app.replication.message.NCLifecycleTaskReportMessage;
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
@@ -37,7 +38,6 @@
 import org.apache.asterix.hyracks.bootstrap.CCApplication;
 import org.apache.asterix.runtime.transaction.ResourceIdManager;
 import org.apache.asterix.runtime.utils.BulkTxnIdFactory;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.asterix.runtime.utils.ClusterStateManager;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.ICCServiceContext;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
index c71e602..e5e33d0 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
@@ -34,6 +34,7 @@
 import org.apache.asterix.active.EntityId;
 import org.apache.asterix.active.NoRetryPolicyFactory;
 import org.apache.asterix.app.active.ActiveNotificationHandler;
+import org.apache.asterix.app.cc.CcApplicationContext;
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.cluster.IClusterStateManager;
@@ -53,7 +54,6 @@
 import org.apache.asterix.metadata.utils.MetadataLockUtil;
 import org.apache.asterix.runtime.functions.FunctionCollection;
 import org.apache.asterix.runtime.functions.FunctionManager;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.asterix.test.active.TestEventsListener.Behavior;
 import org.apache.asterix.test.base.TestMethodTracer;
 import org.apache.asterix.translator.IStatementExecutor;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
index c51fd95..cb123bf 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
@@ -37,6 +37,7 @@
 import org.apache.asterix.app.active.ActiveEntityEventsListener;
 import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.app.cc.CCExtensionManager;
+import org.apache.asterix.app.cc.CcApplicationContext;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.common.exceptions.ErrorCode;
@@ -45,7 +46,6 @@
 import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
 import org.apache.asterix.test.runtime.ExecutionTestUtil;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.SessionOutput;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
index 2ee8fcf..fcc8bc6 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
@@ -86,7 +86,7 @@
                 filterCmpFactories, filterFields, opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory,
                 metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties,
                 true, bloomFilterFields, bloomFilterFalsePositiveRate, index.isPrimaryIndex(), btreeFields,
-                bloomFilterFields != null);
+                bloomFilterFields != null, false);
     }
 
     private static ITypeTraits[] getTypeTraits(MetadataProvider metadataProvider, Dataset dataset, Index index,
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
index 65cbb9e..d68586a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
@@ -27,8 +27,6 @@
 import java.io.StringReader;
 import java.nio.charset.StandardCharsets;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
 import java.util.Map;
 
 import org.apache.asterix.api.java.AsterixJavaClient;
@@ -63,7 +61,7 @@
 
     static {
         EXTENSION_RESULT = "plan.json";
-        PATH_ACTUAL = "target" + File.separator + "jplantest" + SEPARATOR;
+        PATH_ACTUAL = "target" + SEPARATOR + "jplantest" + SEPARATOR;
     }
 
     @Parameters(name = "JsonLogicalPlanTest {index}: {0}")
@@ -71,8 +69,8 @@
         return AbstractOptimizerTest.tests();
     }
 
-    public JsonLogicalPlanTest(final File queryFile, final File expectedFile, final File actualFile) {
-        super(queryFile, expectedFile, actualFile);
+    public JsonLogicalPlanTest(File queryFile, String expectedFilePath, File actualFile) {
+        super(queryFile, actualFile);
     }
 
     @Test
@@ -82,7 +80,7 @@
 
     @Override
     protected void runAndCompare(String query, ILangCompilationProvider provider, Map<String, IAObject> queryParams,
-            IHyracksClientConnection hcc, List<String> linesExpected) throws Exception {
+            IHyracksClientConnection hcc) throws Exception {
         FileUtils.writeStringToFile(actualFile, "", StandardCharsets.UTF_8);
         String planStr;
         try (PrintWriter plan = new PrintWriter(actualFile)) {
@@ -99,7 +97,7 @@
         }
 
         BufferedReader readerActual =
-                new BufferedReader(new InputStreamReader(new FileInputStream(actualFile), "UTF-8"));
+                new BufferedReader(new InputStreamReader(new FileInputStream(actualFile), StandardCharsets.UTF_8));
         String lineActual, objectActual = "";
         boolean firstPlan = false;
         while ((lineActual = readerActual.readLine()) != null) {
@@ -123,10 +121,4 @@
             readerActual.close();
         }
     }
-
-    @Override
-    protected List<String> getExpectedLines() {
-        // this test only checks the produced result is valid, so no expected results
-        return Collections.emptyList();
-    }
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonOptimizedLogicalPlanTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonOptimizedLogicalPlanTest.java
index 7f78743..9bf03ec 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonOptimizedLogicalPlanTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonOptimizedLogicalPlanTest.java
@@ -32,11 +32,11 @@
     static {
         EXTENSION_RESULT = "plan.json";
         optimized = true;
-        PATH_ACTUAL = "target" + File.separator + "joptplantest" + SEPARATOR;
+        PATH_ACTUAL = "target" + SEPARATOR + "joptplantest" + SEPARATOR;
     }
 
-    public JsonOptimizedLogicalPlanTest(File queryFile, File expectedFile, File actualFile) {
-        super(queryFile, expectedFile, actualFile);
+    public JsonOptimizedLogicalPlanTest(File queryFile, String expectedFilePath, File actualFile) {
+        super(queryFile, expectedFilePath, actualFile);
     }
 
     @Parameterized.Parameters(name = "JsonOptimizedLogicalPlanTest {index}: {0}")
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/AbstractOptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/AbstractOptimizerTest.java
index 0fba54b..8126d1e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/AbstractOptimizerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/AbstractOptimizerTest.java
@@ -19,12 +19,10 @@
 package org.apache.asterix.test.optimizer;
 
 import java.io.File;
-import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.List;
 import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -68,12 +66,11 @@
     protected static final String PATH_BASE =
             "src" + SEPARATOR + "test" + SEPARATOR + "resources" + SEPARATOR + "optimizerts" + SEPARATOR;
     protected static final String PATH_QUERIES = PATH_BASE + "queries" + SEPARATOR;
-    protected static final String PATH_EXPECTED = PATH_BASE + "results" + SEPARATOR;
     protected static String PATH_ACTUAL;
 
     protected static final ArrayList<String> ignore = AsterixTestHelper.readTestListFile(FILENAME_IGNORE, PATH_BASE);
     protected static final ArrayList<String> only = AsterixTestHelper.readTestListFile(FILENAME_ONLY, PATH_BASE);
-    protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
+    protected static String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
     protected static final ILangCompilationProvider sqlppCompilationProvider = new SqlppCompilationProvider();
     protected static ILangCompilationProvider extensionLangCompilationProvider = null;
     protected static IStatementExecutorFactory statementExecutorFactory = new DefaultStatementExecutorFactory();
@@ -123,9 +120,8 @@
         }
         if (file.isFile() && file.getName().endsWith(EXTENSION_SQLPP)) {
             String resultFileName = AsterixTestHelper.extToResExt(file.getName(), EXTENSION_RESULT);
-            File expectedFile = new File(PATH_EXPECTED + path + resultFileName);
             File actualFile = new File(PATH_ACTUAL + SEPARATOR + path + resultFileName);
-            testArgs.add(new Object[] { file, expectedFile, actualFile });
+            testArgs.add(new Object[] { file, path + resultFileName, actualFile });
         }
     }
 
@@ -143,20 +139,15 @@
     }
 
     protected final File actualFile;
-    protected final File expectedFile;
     protected final File queryFile;
 
-    public AbstractOptimizerTest(final File queryFile, final File expectedFile, final File actualFile) {
+    public AbstractOptimizerTest(final File queryFile, final File actualFile) {
         this.queryFile = queryFile;
-        this.expectedFile = expectedFile;
         this.actualFile = actualFile;
     }
 
     protected abstract void runAndCompare(String query, ILangCompilationProvider provider,
-            Map<String, IAObject> queryParams, IHyracksClientConnection hcc, List<String> linesExpected)
-            throws Exception;
-
-    protected abstract List<String> getExpectedLines() throws IOException;
+            Map<String, IAObject> queryParams, IHyracksClientConnection hcc) throws Exception;
 
     @Test
     public void test() throws Exception {
@@ -202,11 +193,10 @@
                 provider = extensionLangCompilationProvider;
             }
             IHyracksClientConnection hcc = integrationUtil.getHyracksClientConnection();
-            List<String> linesExpected = getExpectedLines();
             if (repeat) {
-                runAndRepeat(placeholder, substitutions, query, provider, queryParams, hcc, linesExpected);
+                runAndRepeat(placeholder, substitutions, query, provider, queryParams, hcc);
             } else {
-                runAndCompare(query, provider, queryParams, hcc, linesExpected);
+                runAndCompare(query, provider, queryParams, hcc);
             }
 
             LOGGER.info("Test \"" + queryFile.getPath() + "\" PASSED!");
@@ -222,16 +212,16 @@
     }
 
     private void runAndRepeat(String placeholder, JsonNode substitutions, String query,
-            ILangCompilationProvider provider, Map<String, IAObject> queryParams, IHyracksClientConnection hcc,
-            List<String> linesExpected) throws Exception {
+            ILangCompilationProvider provider, Map<String, IAObject> queryParams, IHyracksClientConnection hcc)
+            throws Exception {
         if (substitutions.isArray()) {
             for (int i = 0, size = substitutions.size(); i < size; i++) {
                 String substitute = substitutions.get(i).asText();
                 String newQuery = query.replaceAll(placeholder, substitute);
-                runAndCompare(newQuery, provider, queryParams, hcc, linesExpected);
+                runAndCompare(newQuery, provider, queryParams, hcc);
             }
         } else {
-            runAndCompare(query, provider, queryParams, hcc, linesExpected);
+            runAndCompare(query, provider, queryParams, hcc);
         }
     }
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/CBOOptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/CBOOptimizerTest.java
new file mode 100644
index 0000000..a5a99d9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/CBOOptimizerTest.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.optimizer;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collection;
+import java.util.List;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the optimization tests with CBO enabled. It compares a test's actual result against the expected result in
+ * {@link CBOOptimizerTest#CBO_PATH_EXPECTED} if one is provided. Otherwise, it compares against the expected result in
+ * {@link OptimizerTest#PATH_EXPECTED}.
+ */
+@RunWith(Parameterized.class)
+public class CBOOptimizerTest extends OptimizerTest {
+
+    private static final String CBO_PATH_EXPECTED = PATH_BASE + "results_cbo" + SEPARATOR;
+    static {
+        TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
+        EXTENSION_RESULT = "plan";
+        PATH_ACTUAL = "target" + SEPARATOR + "cbo_opttest" + SEPARATOR;
+    }
+
+    @Parameters(name = "CBOOptimizerTest {index}: {0}")
+    public static Collection<Object[]> tests() {
+        return AbstractOptimizerTest.tests();
+    }
+
+    public CBOOptimizerTest(File queryFile, String expectedFilePath, File actualFile) {
+        super(queryFile, expectedFilePath, actualFile);
+    }
+
+    @Override
+    protected List<String> getExpectedLines() throws IOException {
+        Path cboFilePath = Path.of(CBO_PATH_EXPECTED, expectedFilePath);
+        return Files.exists(cboFilePath) ? Files.readAllLines(cboFilePath, StandardCharsets.UTF_8)
+                : super.getExpectedLines();
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
index a89585d..f02b4d6 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
@@ -24,6 +24,7 @@
 import java.io.StringReader;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
+import java.nio.file.Path;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
@@ -51,12 +52,14 @@
 @RunWith(Parameterized.class)
 public class OptimizerTest extends AbstractOptimizerTest {
 
+    protected static final String PATH_EXPECTED = PATH_BASE + "results" + SEPARATOR;
     private static final String PATTERN_VAR_ID_PREFIX = "\\$\\$";
     private static final Pattern PATTERN_VAR_ID = Pattern.compile(PATTERN_VAR_ID_PREFIX + "(\\d+)");
+    protected String expectedFilePath;
 
     static {
         EXTENSION_RESULT = "plan";
-        PATH_ACTUAL = "target" + File.separator + "opttest" + SEPARATOR;
+        PATH_ACTUAL = "target" + SEPARATOR + "opttest" + SEPARATOR;
     }
 
     @Parameters(name = "OptimizerTest {index}: {0}")
@@ -64,8 +67,9 @@
         return AbstractOptimizerTest.tests();
     }
 
-    public OptimizerTest(final File queryFile, final File expectedFile, final File actualFile) {
-        super(queryFile, expectedFile, actualFile);
+    public OptimizerTest(File queryFile, String expectedFilePath, File actualFile) {
+        super(queryFile, actualFile);
+        this.expectedFilePath = expectedFilePath;
     }
 
     @Test
@@ -75,7 +79,7 @@
 
     @Override
     protected void runAndCompare(String query, ILangCompilationProvider provider, Map<String, IAObject> queryParams,
-            IHyracksClientConnection hcc, List<String> linesExpected) throws Exception {
+            IHyracksClientConnection hcc) throws Exception {
         FileUtils.writeStringToFile(actualFile, "", StandardCharsets.UTF_8);
         try (PrintWriter plan = new PrintWriter(actualFile)) {
             AsterixJavaClient asterix = new AsterixJavaClient(
@@ -89,6 +93,7 @@
 
         List<String> linesActual = Files.readAllLines(actualFile.toPath(), StandardCharsets.UTF_8);
 
+        List<String> linesExpected = getExpectedLines();
         int varBaseExpected = findBaseVarId(linesExpected);
         int varBaseActual = findBaseVarId(linesActual);
 
@@ -116,9 +121,8 @@
         }
     }
 
-    @Override
     protected List<String> getExpectedLines() throws IOException {
-        return Files.readAllLines(expectedFile.toPath(), StandardCharsets.UTF_8);
+        return Files.readAllLines(Path.of(PATH_EXPECTED, expectedFilePath), StandardCharsets.UTF_8);
     }
 
     private boolean planLineEquals(String lineExpected, int varIdBaseExpected, String lineActual, int varIdBaseActual) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/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/resources/cc-batch-lookup.conf b/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf
new file mode 100644
index 0000000..6e10481
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-batch-lookup.conf
@@ -0,0 +1,65 @@
+; Licensed to the Apache Software Foundation (ASF) under one
+; or more contributor license agreements.  See the NOTICE file
+; distributed with this work for additional information
+; regarding copyright ownership.  The ASF licenses this file
+; to you under the Apache License, Version 2.0 (the
+; "License"); you may not use this file except in compliance
+; with the License.  You may obtain a copy of the License at
+;
+;   http://www.apache.org/licenses/LICENSE-2.0
+;
+; Unless required by applicable law or agreed to in writing,
+; software distributed under the License is distributed on an
+; "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+; KIND, either express or implied.  See the License for the
+; specific language governing permissions and limitations
+; under the License.
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=target/tmp/asterix_nc1/iodevice1,
+iodevices=../asterix-server/target/tmp/asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+credential.file=src/test/resources/security/passwd
+python.cmd.autolocate=true
+python.env=FOO=BAR=BAZ,BAR=BAZ
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.pagesize=32KB
+storage.buffercache.size=128MB
+storage.memorycomponent.globalbudget=512MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+credential.file=src/test/resources/security/passwd
+
+[common]
+log.dir = logs/
+log.level = INFO
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.textsearchmemory=160KB
+compiler.windowmemory=192KB
+compiler.batch.lookup=true
+messaging.frame.size=4096
+messaging.frame.count=512
+
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query4.sqlpp
new file mode 100644
index 0000000..fae9ea4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query4.sqlpp
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: ASTERIXDB-3043. Part of q18 of tpcds.
+ */
+
+DROP DATAVERSE tpcds IF EXISTS;
+CREATE DATAVERSE tpcds;
+
+USE tpcds;
+
+CREATE TYPE tpcds.catalog_sales_type AS
+ CLOSED {
+  cs_sold_date_sk:           bigint?,
+  cs_sold_time_sk:           bigint?,
+  cs_ship_date_sk:           bigint?,
+  cs_bill_customer_sk:       bigint?,
+  cs_bill_cdemo_sk:          bigint?,
+  cs_bill_hdemo_sk:          bigint?,
+  cs_bill_addr_sk:           bigint?,
+  cs_ship_customer_sk:       bigint?,
+  cs_ship_cdemo_sk:          bigint?,
+  cs_ship_hdemo_sk:          bigint?,
+  cs_ship_addr_sk:           bigint?,
+  cs_call_center_sk:         bigint?,
+  cs_catalog_page_sk:        bigint?,
+  cs_ship_mode_sk:           bigint?,
+  cs_warehouse_sk:           bigint?,
+  cs_item_sk:                bigint,
+  cs_promo_sk:               bigint?,
+  cs_order_number:           bigint,
+  cs_quantity:               bigint?,
+  cs_wholesale_cost:         double?,
+  cs_list_price:             double?,
+  cs_sales_price:            double?,
+  cs_ext_discount_amt:       double?,
+  cs_ext_sales_price:        double?,
+  cs_ext_wholesale_cost:     double?,
+  cs_ext_list_price:         double?,
+  cs_ext_tax:                double?,
+  cs_coupon_amt:             double?,
+  cs_ext_ship_cost:          double?,
+  cs_net_paid:               double?,
+  cs_net_paid_inc_tax:       double?,
+  cs_net_paid_inc_ship:      double?,
+  cs_net_paid_inc_ship_tax:  double?,
+  cs_net_profit:             double?
+};
+
+CREATE TYPE tpcds.customer_demographics_type AS
+ CLOSED {
+  cd_demo_sk : int64,
+  cd_gender : string?,
+  cd_marital_status : string?,
+  cd_education_status : string?,
+  cd_purchase_estimate : int64?,
+  cd_credit_rating : string?,
+  cd_dep_count : int64?,
+  cd_dep_employed_count : int64?,
+  cd_dep_college_count : int64?
+};
+
+CREATE TYPE tpcds.customer_type AS
+ CLOSED {
+  c_customer_sk : int64,
+  c_customer_id : string,
+  c_current_cdemo_sk : int64?,
+  c_current_hdemo_sk : int64?,
+  c_current_addr_sk : int64?,
+  c_first_shipto_date_sk : int64?,
+  c_first_sales_date_sk : int64?,
+  c_salutation : string?,
+  c_first_name : string?,
+  c_last_name : string?,
+  c_preferred_cust_flag : string?,
+  c_birth_day : int64?,
+  c_birth_month : int64?,
+  c_birth_year : int64?,
+  c_birth_country : string?,
+  c_login : string?,
+  c_email_address : string?,
+  c_last_review_date : string?
+};
+
+CREATE TYPE tpcds.customer_address_type AS
+ CLOSED {
+  ca_address_sk : bigint,
+  ca_address_id : string,
+  ca_street_number : string?,
+  ca_street_name : string?,
+  ca_street_type : string?,
+  ca_suite_number : string?,
+  ca_city : string?,
+  ca_county : string?,
+  ca_state : string?,
+  ca_zip : string?,
+  ca_country : string?,
+  ca_gmt_offset : double?,
+  ca_location_type : string?
+ };
+
+CREATE TYPE tpcds.item_type AS
+ CLOSED {
+  i_item_sk : bigint,
+  i_item_id : string,
+  i_rec_start_date : string?,
+  i_rec_end_date : string?,
+  i_item_desc : string?,
+  i_current_price : double?,
+  i_wholesale_cost : double?,
+  i_brand_id : bigint?,
+  i_brand : string?,
+  i_class_id : bigint?,
+  i_class : string?,
+  i_category_id : bigint?,
+  i_category : string?,
+  i_manufact_id : bigint?,
+  i_manufact : string?,
+  i_size : string?,
+  i_formulation : string?,
+  i_color : string?,
+  i_units : string?,
+  i_container : string?,
+  i_manager_id : bigint?,
+  i_product_name : string?
+};
+
+CREATE TYPE tpcds.date_dim_type AS
+ CLOSED {
+  d_date_sk : bigint,
+  d_date_id : string,
+  d_date : string?,
+  d_month_seq : bigint?,
+  d_week_seq : bigint?,
+  d_quarter_seq : bigint?,
+  d_year : bigint? ,
+  d_dow : bigint? ,
+  d_moy : bigint?,
+  d_dom : bigint?,
+  d_qoy : bigint?,
+  d_fy_year : bigint?,
+  d_fy_quarter_seq : bigint?,
+  d_fy_week_seq : bigint?,
+  d_day_name : string?,
+  d_quarter_name : string?,
+  d_holiday : string?,
+  d_weekend : string?,
+  d_following_holiday : string?,
+  d_first_dom : bigint?,
+  d_last_dom : bigint?,
+  d_same_day_ly : bigint?,
+  d_same_day_lq : bigint?,
+  d_current_day : string?,
+  d_current_week : string?,
+  d_current_month : string?,
+  d_current_quarter : string?,
+  d_current_year : string?
+};
+
+CREATE DATASET catalog_sales(catalog_sales_type) PRIMARY KEY cs_item_sk, cs_order_number;
+
+CREATE DATASET customer_demographics(customer_demographics_type) PRIMARY KEY cd_demo_sk;
+
+CREATE DATASET customer(customer_type) PRIMARY KEY c_customer_sk;
+
+CREATE DATASET customer_address(customer_address_type) PRIMARY KEY ca_address_sk;
+
+CREATE DATASET item(item_type) PRIMARY KEY i_item_sk;
+
+CREATE DATASET date_dim(date_dim_type) PRIMARY KEY d_date_sk;
+
+SELECT count (*)
+FROM customer c, customer_demographics cd2, customer_address ca
+WHERE
+ c.c_current_cdemo_sk  /*+ indexnl */ = cd2.cd_demo_sk
+ AND c.c_current_addr_sk  /*+ indexnl */= ca.ca_address_sk
+ AND c.c_birth_month in [4,5];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query4.plan
new file mode 100644
index 0000000..4e1a931
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query4.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- HYBRID_HASH_JOIN [$$57][$$70]  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH (tpcds.customer_address.customer_address)  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$74]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (tpcds.customer_demographics.customer_demographics)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$72]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (tpcds.customer)  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- UNNEST  |UNPARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/only_batch_lookup.xml b/asterixdb/asterix-app/src/test/resources/runtimets/only_batch_lookup.xml
new file mode 100644
index 0000000..334dd52
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/only_batch_lookup.xml
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+  <test-group name="failed">
+  </test-group>
+</test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.1.ddl.sqlpp
new file mode 100644
index 0000000..c0aadca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.1.ddl.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+use test1;
+
+create type t1 as closed {
+  c1 : bigint,
+  c2 : bigint
+};
+
+create dataset ds1(t1) primary key c1;
+
+create index idx1 on ds1(c2);
+
+create view v1 as select value v from ds1 v;
+
+create synonym s1 for v1;
+
+create function f1() {
+  select "Dataverse" as k, DataverseName as dv, DataverseName as n from Metadata.`Dataverse`
+  union all
+  select "Datatype" as k, DataverseName as dv, DatatypeName as n from Metadata.`Datatype`
+  union all
+  select "Dataset" as k, DataverseName as dv, DatasetName as n from Metadata.`Dataset`
+  union all
+  select "Index" as k, DataverseName as dv, IndexName as n from Metadata.`Index` where not(isPrimary)
+  union all
+  select "Synonym" as k, DataverseName as dv, SynonymName as n from Metadata.`Synonym`
+  union all
+  select "Function" as k, DataverseName as dv, Name as n from Metadata.`Function`
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.10.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.10.query.sqlpp
index 6095b26..53097a4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.10.query.sqlpp
@@ -17,8 +17,10 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Check that objects were not created
+ */
 
-select value v from range(1,2) v where v > ?;
+select count(*) cnt
+from test1.f1() t
+where dv like "test2";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.2.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.2.query.sqlpp
index 6095b26..1e3fb34 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.2.query.sqlpp
@@ -17,8 +17,7 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
-
-select value v from range(1,2) v where v > ?;
+select k, n
+from test1.f1() t
+where dv like "test1"
+order by k, n;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.3.ddl.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.3.ddl.sqlpp
index 6095b26..b913179 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.3.ddl.sqlpp
@@ -17,8 +17,29 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test that DROP DDL statements are not executed in compile-only mode
+ */
 
-select value v from range(1,2) v where v > ?;
+-- param compile-only:string=true
+
+drop function test1.f1();
+
+drop synonym test1.s1;
+
+drop view test1.v1;
+
+drop index test1.ds1.idx1;
+
+drop dataset test1.ds1;
+
+drop type test1.t1;
+
+
+
+
+
+
+
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.4.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.4.query.sqlpp
index 6095b26..5760b8e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.4.query.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Check that objects were not dropped
+ */
 
-select value v from range(1,2) v where v > ?;
+select k, n
+from test1.f1() t
+where dv like "test1"
+order by k, n;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.5.ddl.sqlpp
similarity index 87%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.5.ddl.sqlpp
index 6095b26..37146d7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.5.ddl.sqlpp
@@ -17,8 +17,10 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test that DROP DDL statements are not executed in compile-only mode
+ */
 
-select value v from range(1,2) v where v > ?;
+-- param compile-only:string=true
+
+drop dataverse test1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.6.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.6.query.sqlpp
index 6095b26..5760b8e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.6.query.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Check that objects were not dropped
+ */
 
-select value v from range(1,2) v where v > ?;
+select k, n
+from test1.f1() t
+where dv like "test1"
+order by k, n;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.7.ddl.sqlpp
similarity index 70%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.7.ddl.sqlpp
index 6095b26..9b41631 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.7.ddl.sqlpp
@@ -17,8 +17,27 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test that CREATE DDL statements are not executed in compile-only mode
+ */
 
-select value v from range(1,2) v where v > ?;
+-- param compile-only:string=true
+
+use test1;
+
+create type t2 as closed {
+  c1 : bigint,
+  c2 : bigint
+};
+
+create dataset ds2(t2) primary key c1;
+
+create index idx2 on ds2(c2);
+
+create view v2 as select value v from ds2 v;
+
+create synonym s2 for v2;
+
+create function f2() {
+  select value v from v2 v
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.8.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.8.query.sqlpp
index 6095b26..c5afa49 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.8.query.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Check that objects were not created
+ */
 
-select value v from range(1,2) v where v > ?;
+select k, n
+from test1.f1() t
+where dv like "test1"
+order by k, n;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.9.ddl.sqlpp
similarity index 85%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.9.ddl.sqlpp
index 6095b26..7fb26db 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly-2/compileonly-2.9.ddl.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test that CREATE DDL statements are not executed in compile-only mode
+ */
 
-select value v from range(1,2) v where v > ?;
+-- param compile-only:string=true
+
+drop dataverse test2 if exists;
+create dataverse test2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.ddl.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.ddl.sqlpp
index 6095b26..82e9f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.ddl.sqlpp
@@ -17,8 +17,7 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+drop dataverse test1 if exists;
+create dataverse test1;
 
-select value v from range(1,2) v where v > ?;
+create dataset test1.t1(c1 int not unknown, c2 int) primary key c1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.2.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.2.plans.sqlpp
index 2a4952f..6095b26 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.2.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.2.plans.sqlpp
@@ -17,11 +17,8 @@
  * under the License.
  */
 
-/*
- * Test additional information returned when client-type=jdbc
- */
-
--- param client-type:string=jdbc
 -- param compile-only:string=true
+-- param logical-plan:string=true
+-- param plan-format:string=json
 
-select v from range(1,2) v where v between ? and ? ;
+select value v from range(1,2) v where v > ?;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.3.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.3.plans.sqlpp
index d7217a4..2a4952f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.3.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.3.plans.sqlpp
@@ -18,10 +18,10 @@
  */
 
 /*
- * Test additional information returned when client-type=jdbc (with explain)
+ * Test additional information returned when client-type=jdbc
  */
 
 -- param client-type:string=jdbc
 -- param compile-only:string=true
 
-explain select v from range(1,2) v where v between ? and ? ;
+select v from range(1,2) v where v between ? and ? ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.4.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.4.plans.sqlpp
index 8d2bd74..d7217a4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.4.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.4.plans.sqlpp
@@ -18,16 +18,10 @@
  */
 
 /*
- * Test additional information returned when client-type=jdbc (update statement)
+ * Test additional information returned when client-type=jdbc (with explain)
  */
 
 -- param client-type:string=jdbc
 -- param compile-only:string=true
 
-drop dataverse test1 if exists;
-
-create dataverse test1;
-
-create dataset test1.t1(c1 int not unknown, c2 int) primary key c1;
-
-insert into test1.t1 [{"c1": 1, "c2": ? }, {"c1": 3, "c2": ? }];
+explain select v from range(1,2) v where v between ? and ? ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.5.plans.sqlpp
similarity index 81%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.5.plans.sqlpp
index 6095b26..5947c5d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.5.plans.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Test additional information returned when client-type=jdbc (update statement)
+ */
 
-select value v from range(1,2) v where v > ?;
+-- param client-type:string=jdbc
+-- param compile-only:string=true
+
+insert into test1.t1 [{"c1": 1, "c2": ? }, {"c1": 3, "c2": ? }];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.1.ddl.sqlpp
new file mode 100644
index 0000000..e1d6b10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Test ANALYZE DATASET statement
+ */
+
+set `import-private-functions` `true`;
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create function listMetadata(showSourceAvgItemSize, showSeed) {
+  select i.DatasetName, i.IndexName, i.SampleCardinalityTarget, i.SourceCardinality,
+    case when showSourceAvgItemSize then i.SourceAvgItemSize else i.SourceAvgItemSize > 0 end as SourceAvgItemSize,
+    case when showSeed then i.SampleSeed else i.SampleSeed is known end as SampleSeed
+  from Metadata.`Index` i
+  where i.DataverseName = "test" and i.IndexName like "sample_idx%"
+  order by i.IndexName
+};
+
+create function showSampleStats(dsName, idxName, showMinMax) {
+  select count(v.values) as cnt,
+    case when showMinMax then min(v.values[0]) else min(v.values[0]) > 0 end as min_pk,
+    case when showMinMax then max(v.values[0]) else max(v.values[0]) > 0 end as max_pk,
+    case when showMinMax then min(v.values[1].x) else min(v.values[1].x) < 0 end as min_x,
+    case when showMinMax then max(v.values[1].x) else max(v.values[1].x) < 0 end as max_x
+  from dump_index("test", dsName, idxName) as v
+};
+
+create type t1 as open {
+  id : bigint
+};
+
+create dataset ds1(t1) primary key id;
+
+-- analyze on an empty dataset
+
+analyze dataset ds1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.10.ddl.sqlpp
similarity index 78%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.10.ddl.sqlpp
index 6095b26..da5fe13 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.10.ddl.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: analyze dataset with sample=medium.
+ * Note, there are fewer tuples in the dataset that the target sample size
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+analyze dataset test.ds1 with { "sample": "medium", "sample-seed": 234.0 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.11.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.11.query.sqlpp
index 6095b26..38ded0a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.11.query.sqlpp
@@ -17,8 +17,14 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was re-created
+ */
 
-select value v from range(1,2) v where v > ?;
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+  listMetadata(false, true) metadata,
+  showSampleStats("ds1", "sample_idx_2_ds1", true) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.12.update.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.12.update.sqlpp
index 6095b26..0e3886d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.12.update.sqlpp
@@ -17,8 +17,12 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: Insert more data
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+insert into ds1
+select element {'id':x, 'x':-x }
+from range(1101, 4400) as x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.13.ddl.sqlpp
similarity index 80%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.13.ddl.sqlpp
index 6095b26..b098016 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.13.ddl.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: analyze dataset with sample=medium
+ * Note, there are more tuples in the dataset that the target sample size
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+analyze dataset ds1 with { "sample": "medium" };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.14.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.14.query.sqlpp
index 6095b26..4cae202 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.14.query.sqlpp
@@ -17,8 +17,14 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was re-created
+ */
 
-select value v from range(1,2) v where v > ?;
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+  listMetadata(false, false) metadata,
+  showSampleStats("ds1", "sample_idx_1_ds1", false) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.15.ddl.sqlpp
similarity index 78%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.15.ddl.sqlpp
index 6095b26..6ceb8141 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.15.ddl.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: analyze dataset with sample=high.
+ * Note, there are fewer tuples in the dataset that the target sample size
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+analyze dataset ds1 with { "sample": "high", "sample-seed": "345" };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.16.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.16.query.sqlpp
index 6095b26..38ded0a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.16.query.sqlpp
@@ -17,8 +17,14 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was re-created
+ */
 
-select value v from range(1,2) v where v > ?;
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+  listMetadata(false, true) metadata,
+  showSampleStats("ds1", "sample_idx_2_ds1", true) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.17.update.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.17.update.sqlpp
index 6095b26..0c5f8ad 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.17.update.sqlpp
@@ -17,8 +17,12 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: Insert more data
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+insert into ds1
+select element {'id':x, 'x':-x }
+from range(4401, 17100) as x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.18.ddl.sqlpp
similarity index 80%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.18.ddl.sqlpp
index 6095b26..4385f95 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.18.ddl.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: analyze dataset with sample=high
+ * Note, there are more tuples in the dataset that the target sample size
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+analyze dataset ds1 with { "sample": "high" };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.19.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.19.query.sqlpp
index 6095b26..4cae202 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.19.query.sqlpp
@@ -17,8 +17,14 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was re-created
+ */
 
-select value v from range(1,2) v where v > ?;
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+  listMetadata(false, false) metadata,
+  showSampleStats("ds1", "sample_idx_1_ds1", false) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.2.query.sqlpp
similarity index 83%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.2.query.sqlpp
index 6095b26..e786e0e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.2.query.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: Check that the sample index was created even thought
+ *              the source dataset is empty
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+listMetadata(true, false);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.20.ddl.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.20.ddl.sqlpp
index 6095b26..ffa19b4a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.20.ddl.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: test that the sample index is dropped when
+ *              its source dataset is dropped
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+drop dataset ds1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.21.query.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.21.query.sqlpp
index 6095b26..759fc3f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.21.query.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was dropped
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+select count(*) cnt
+from listMetadata(true, false) v;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.3.update.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.3.update.sqlpp
index 6095b26..1b83b99 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.3.update.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+use test;
 
-select value v from range(1,2) v where v > ?;
+insert into ds1
+select element {'id':x, 'x':-x }
+from range(1, 8) as x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.4.ddl.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.4.ddl.sqlpp
index 6095b26..ed97897 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.4.ddl.sqlpp
@@ -17,8 +17,8 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: Test sample size parameter
+ */
 
-select value v from range(1,2) v where v > ?;
+analyze dataset test.ds1 with { "sample": "low", "sample-seed": 123 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.5.query.sqlpp
similarity index 72%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.5.query.sqlpp
index 6095b26..e0cd6cc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.5.query.sqlpp
@@ -17,8 +17,15 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that sample index was recreated with a new name.
+ * Also check that all 8 dataset tuples are in the sample
+ */
 
-select value v from range(1,2) v where v > ?;
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+  listMetadata(false, true) metadata,
+  showSampleStats("ds1", "sample_idx_2_ds1", true) stats
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.6.update.sqlpp
similarity index 84%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.6.update.sqlpp
index 6095b26..340fbb3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.6.update.sqlpp
@@ -17,8 +17,12 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: Insert more data
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+insert into ds1
+select element {'id':x, 'x':-x }
+from range(9, 1100) as x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.7.query.sqlpp
similarity index 72%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.7.query.sqlpp
index 6095b26..c4930b0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.7.query.sqlpp
@@ -17,8 +17,15 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample content remains the same
+ * after INSERT because we did not run ANALYZE DATASET after that
+ */
 
-select value v from range(1,2) v where v > ?;
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+  listMetadata(false, true) metadata,
+  showSampleStats("ds1", "sample_idx_2_ds1", true) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.8.ddl.sqlpp
similarity index 80%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.8.ddl.sqlpp
index 6095b26..2f452f0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.8.ddl.sqlpp
@@ -17,8 +17,11 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: analyze dataset with sample=low.
+ * Note, there are more tuples in the dataset that the target sample size
+ */
 
-select value v from range(1,2) v where v > ?;
+use test;
+
+analyze dataset ds1 with { "sample": "low" };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.9.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.9.query.sqlpp
index 6095b26..4cae202 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/analyze-dataset-1/analyze-dataset-1.9.query.sqlpp
@@ -17,8 +17,14 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+/*
+ * Description: check that the sample was re-created
+ */
 
-select value v from range(1,2) v where v > ?;
+set `import-private-functions` `true`;
+
+use test;
+
+select * from
+  listMetadata(false, false) metadata,
+  showSampleStats("ds1", "sample_idx_1_ds1", false) stats;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.4.query.sqlpp
similarity index 72%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.4.query.sqlpp
index 6095b26..1f7c7d5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.4.query.sqlpp
@@ -17,8 +17,14 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+// see ASTERIXDB-3043
+SET `compiler.joinmemory` "160KB";
 
-select value v from range(1,2) v where v > ?;
+USE tpcds;
+
+SELECT count (*) AS cnt
+FROM customer c, customer_demographics cd2, customer_address ca
+WHERE
+ c.c_current_cdemo_sk /*+ indexnl */ = cd2.cd_demo_sk
+ AND c.c_current_addr_sk /*+ indexnl */ = ca.ca_address_sk
+ AND c.c_birth_month in [4,5];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.5.query.sqlpp
similarity index 74%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.5.query.sqlpp
index 6095b26..3a7ae0b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q18/q18.5.query.sqlpp
@@ -17,8 +17,14 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+// see ASTERIXDB-3043
+SET `compiler.joinmemory` "160KB";
 
-select value v from range(1,2) v where v > ?;
+USE tpcds;
+
+SELECT count (*) AS cnt
+FROM customer c, customer_demographics cd2, customer_address ca
+WHERE
+ c.c_current_cdemo_sk = cd2.cd_demo_sk
+ AND c.c_current_addr_sk = ca.ca_address_sk
+ AND c.c_birth_month in [4,5];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
index c55c0bc..54793dc 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\.external\.field\.pushdown" : true,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
index 661daf3..06da0ad 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\.external\.field\.pushdown" : true,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
index 1f0e865..bef14c1 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\.external\.field\.pushdown" : true,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.10.adm
new file mode 100644
index 0000000..bacb60c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.10.adm
@@ -0,0 +1 @@
+{ "cnt": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.2.adm
new file mode 100644
index 0000000..c1ff5e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.2.adm
@@ -0,0 +1,6 @@
+{ "k": "Dataset", "n": "ds1" }
+{ "k": "Dataset", "n": "v1" }
+{ "k": "Datatype", "n": "t1" }
+{ "k": "Dataverse", "n": "test1" }
+{ "k": "Function", "n": "f1" }
+{ "k": "Synonym", "n": "s1" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.4.adm
new file mode 100644
index 0000000..c1ff5e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.4.adm
@@ -0,0 +1,6 @@
+{ "k": "Dataset", "n": "ds1" }
+{ "k": "Dataset", "n": "v1" }
+{ "k": "Datatype", "n": "t1" }
+{ "k": "Dataverse", "n": "test1" }
+{ "k": "Function", "n": "f1" }
+{ "k": "Synonym", "n": "s1" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.6.adm
new file mode 100644
index 0000000..c1ff5e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.6.adm
@@ -0,0 +1,6 @@
+{ "k": "Dataset", "n": "ds1" }
+{ "k": "Dataset", "n": "v1" }
+{ "k": "Datatype", "n": "t1" }
+{ "k": "Dataverse", "n": "test1" }
+{ "k": "Function", "n": "f1" }
+{ "k": "Synonym", "n": "s1" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.8.adm
new file mode 100644
index 0000000..c1ff5e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly-2/compileonly-2.8.adm
@@ -0,0 +1,6 @@
+{ "k": "Dataset", "n": "ds1" }
+{ "k": "Dataset", "n": "v1" }
+{ "k": "Datatype", "n": "t1" }
+{ "k": "Dataverse", "n": "test1" }
+{ "k": "Function", "n": "f1" }
+{ "k": "Synonym", "n": "s1" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.adm
deleted file mode 100644
index 1ced420..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.adm
+++ /dev/null
@@ -1 +0,0 @@
-{"statementCategory":"query","statementParameters":[1,2]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.1.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.regexjson
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.1.regexjson
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.regexjson
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.3.adm
index 83b47f4..1ced420 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.3.adm
@@ -1 +1 @@
-{"statementCategory":"query","statementParameters":[1,2],"explainOnly":true}
\ No newline at end of file
+{"statementCategory":"query","statementParameters":[1,2]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.4.adm
index 63c482a..83b47f4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.4.adm
@@ -1 +1 @@
-{"statementCategory":"update","statementParameters":[1,2]}
\ No newline at end of file
+{"statementCategory":"query","statementParameters":[1,2],"explainOnly":true}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.5.adm
new file mode 100644
index 0000000..63c482a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.5.adm
@@ -0,0 +1 @@
+{"statementCategory":"update","statementParameters":[1,2]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.11.adm
new file mode 100644
index 0000000..58f454b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.11.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_2_ds1", "SampleCardinalityTarget": 4252, "SourceCardinality": 1100, "SourceAvgItemSize": true, "SampleSeed": 234 }, "stats": { "cnt": 1100, "min_pk": 1, "max_pk": 1100, "min_x": -1100, "max_x": -1 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.14.adm
new file mode 100644
index 0000000..6ef756a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.14.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_1_ds1", "SampleCardinalityTarget": 4252, "SourceCardinality": 4400, "SourceAvgItemSize": true, "SampleSeed": true }, "stats": { "cnt": 4246, "min_pk": true, "max_pk": true, "min_x": true, "max_x": true } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.16.adm
new file mode 100644
index 0000000..01eb5b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.16.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_2_ds1", "SampleCardinalityTarget": 17008, "SourceCardinality": 4400, "SourceAvgItemSize": true, "SampleSeed": 345 }, "stats": { "cnt": 4400, "min_pk": 1, "max_pk": 4400, "min_x": -4400, "max_x": -1 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.19.adm
new file mode 100644
index 0000000..60b969f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.19.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_1_ds1", "SampleCardinalityTarget": 17008, "SourceCardinality": 17100, "SourceAvgItemSize": true, "SampleSeed": true }, "stats": { "cnt": 16972, "min_pk": true, "max_pk": true, "min_x": true, "max_x": true } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.2.adm
new file mode 100644
index 0000000..e3cefee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.2.adm
@@ -0,0 +1 @@
+{ "DatasetName": "ds1", "IndexName": "sample_idx_1_ds1", "SampleCardinalityTarget": 1063, "SourceCardinality": 0, "SourceAvgItemSize": 0, "SampleSeed": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.21.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.21.adm
new file mode 100644
index 0000000..bacb60c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.21.adm
@@ -0,0 +1 @@
+{ "cnt": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.5.adm
new file mode 100644
index 0000000..605bb12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.5.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_2_ds1", "SampleCardinalityTarget": 1063, "SourceCardinality": 8, "SourceAvgItemSize": true, "SampleSeed": 123 }, "stats": { "cnt": 8, "min_pk": 1, "max_pk": 8, "min_x": -8, "max_x": -1 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.7.adm
new file mode 100644
index 0000000..605bb12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.7.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_2_ds1", "SampleCardinalityTarget": 1063, "SourceCardinality": 8, "SourceAvgItemSize": true, "SampleSeed": 123 }, "stats": { "cnt": 8, "min_pk": 1, "max_pk": 8, "min_x": -8, "max_x": -1 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.9.adm
new file mode 100644
index 0000000..0084d2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/analyze-dataset-1/analyze-dataset-1.9.adm
@@ -0,0 +1 @@
+{ "metadata": { "DatasetName": "ds1", "IndexName": "sample_idx_1_ds1", "SampleCardinalityTarget": 1063, "SourceCardinality": 1100, "SourceAvgItemSize": true, "SampleSeed": true }, "stats": { "cnt": 1033, "min_pk": true, "max_pk": true, "min_x": true, "max_x": true } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-01/explain_object_constructor-01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-01/explain_object_constructor-01.1.adm
index 9a536a6..e89d96a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-01/explain_object_constructor-01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-01/explain_object_constructor-01.1.adm
@@ -2,7 +2,7 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    assign [$$2] <- [{ a: "1", b: 1 }]
+    assign [$$2] <- [{ "a": "1", "b": 1 }]
     -- ASSIGN  |UNPARTITIONED|
       empty-tuple-source
       -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-02/explain_object_constructor-02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-02/explain_object_constructor-02.1.adm
index c637d87..2a8a531 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-02/explain_object_constructor-02.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_object_constructor-02/explain_object_constructor-02.1.adm
@@ -2,7 +2,7 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    assign [$$2] <- [{ a: "1" }]
+    assign [$$2] <- [{ "a": "1" }]
     -- ASSIGN  |UNPARTITIONED|
       empty-tuple-source
-      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.adm
index de214d0..02a5277 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.adm
@@ -15,7 +15,7 @@
               subplan {
                         aggregate [$$39] <- [empty-stream()]
                         -- AGGREGATE  |LOCAL|
-                          select (not(if-missing-or-null(eq($$48, "string"), FALSE)))
+                          select (not(if-missing-or-null(eq($$48, "string"), false)))
                           -- STREAM_SELECT  |LOCAL|
                             assign [$$48] <- [$$ht.getField("display_url")]
                             -- ASSIGN  |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.adm
index 9f122c6..f0a3dcb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.adm
@@ -15,7 +15,7 @@
               subplan {
                         aggregate [$$39] <- [empty-stream()]
                         -- AGGREGATE  |LOCAL|
-                          select (not(if-missing-or-null(eq($$48, "string"), FALSE)))
+                          select (not(if-missing-or-null(eq($$48, "string"), false)))
                           -- STREAM_SELECT  |LOCAL|
                             assign [$$48] <- [$$ht.getField("display_url")]
                             -- ASSIGN  |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.adm
index 4a602ef..6114e6d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.adm
@@ -4,7 +4,7 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     project ([$$20])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$20] <- [switch-case(TRUE, is-array($$22), $$24, $$25)]
+      assign [$$20] <- [switch-case(true, is-array($$22), $$24, $$25)]
       -- ASSIGN  |PARTITIONED|
         project ([$$22, $$24, $$25])
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.adm
index 56dd2ee..420abf4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.adm
@@ -4,7 +4,7 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     project ([$$20])
     -- STREAM_PROJECT  |PARTITIONED|
-      assign [$$20] <- [switch-case(TRUE, is-array($$22), $$24, $$25)]
+      assign [$$20] <- [switch-case(true, is-array($$22), $$24, $$25)]
       -- ASSIGN  |PARTITIONED|
         project ([$$22, $$24, $$25])
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.adm
index 30d5ac3..cfe8897 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.adm
@@ -93,7 +93,7 @@
                                                   -- HASH_PARTITION_EXCHANGE [$$81]  |PARTITIONED|
                                                     project ([$$104, $$81])
                                                     -- STREAM_PROJECT  |PARTITIONED|
-                                                      assign [$$104, $$81] <- [TRUE, $$ht2.getField("text")]
+                                                      assign [$$104, $$81] <- [true, $$ht2.getField("text")]
                                                       -- ASSIGN  |PARTITIONED|
                                                         project ([$$ht2])
                                                         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/type_validation/type_validation.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/type_validation/type_validation.1.adm
index f164f97..121262f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/type_validation/type_validation.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/type_validation/type_validation.1.adm
@@ -1 +1 @@
-"907 9.07 \"907\" 9.07 TRUE date: { 2013-01-01 } datetime: { 1989-09-07T12:13:14.039 } "
+"907 9.07 \"907\" 9.07 true date: { 2013-01-01 } datetime: { 1989-09-07T12:13:14.039 } "
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.adm
index 3543f5d..607ed1a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.adm
@@ -12,7 +12,7 @@
           -- STREAM_LIMIT  |PARTITIONED|
             exchange
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              unnest-map [$$17, $$18, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$23, $$24, 2, $$23, $$24, TRUE, TRUE, TRUE) condition (and(lt($$c.getField(2), 150), lt($$c.getField(5), 10000))) limit 10
+              unnest-map [$$17, $$18, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$23, $$24, 2, $$23, $$24, true, true, true) condition (and(lt($$c.getField(2), 150), lt($$c.getField(5), 10000))) limit 10
               -- BTREE_SEARCH  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -24,11 +24,11 @@
                       -- STREAM_PROJECT  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          unnest-map [$$22, $$23, $$24] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$21, TRUE, FALSE, FALSE)
+                          unnest-map [$$22, $$23, $$24] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$21, true, false, false)
                           -- BTREE_SEARCH  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               assign [$$21] <- [150]
                               -- ASSIGN  |PARTITIONED|
                                 empty-tuple-source
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.adm
index 1f4ef66..cad5cf0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.5.adm
@@ -18,7 +18,7 @@
                 -- ASSIGN  |PARTITIONED|
                   exchange
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    unnest-map [$$21, $$22, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$26, $$27, 2, $$26, $$27, TRUE, TRUE, TRUE) condition (lt($$c.getField(2), 150)) limit 5
+                    unnest-map [$$21, $$22, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$26, $$27, 2, $$26, $$27, true, true, true) condition (lt($$c.getField(2), 150)) limit 5
                     -- BTREE_SEARCH  |PARTITIONED|
                       exchange
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -30,11 +30,11 @@
                             -- STREAM_PROJECT  |PARTITIONED|
                               exchange
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                unnest-map [$$25, $$26, $$27] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$24, TRUE, FALSE, FALSE)
+                                unnest-map [$$25, $$26, $$27] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$24, true, false, false)
                                 -- BTREE_SEARCH  |PARTITIONED|
                                   exchange
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     assign [$$24] <- [150]
                                     -- ASSIGN  |PARTITIONED|
                                       empty-tuple-source
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.adm
index a0a3c84..60dedd5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.adm
@@ -12,7 +12,7 @@
           -- STREAM_LIMIT  |PARTITIONED|
             exchange
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              unnest-map [$$14, $$15, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$19, $$20, 2, $$19, $$20, TRUE, TRUE, TRUE) condition (lt($$c.getField(2), 150)) limit 10
+              unnest-map [$$14, $$15, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$19, $$20, 2, $$19, $$20, true, true, true) condition (lt($$c.getField(2), 150)) limit 10
               -- BTREE_SEARCH  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -24,11 +24,11 @@
                       -- STREAM_PROJECT  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          unnest-map [$$18, $$19, $$20] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$17, TRUE, FALSE, FALSE)
+                          unnest-map [$$18, $$19, $$20] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$17, true, false, false)
                           -- BTREE_SEARCH  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               assign [$$17] <- [150]
                               -- ASSIGN  |PARTITIONED|
                                 empty-tuple-source
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
index 44507f4..717f424 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
@@ -12,7 +12,7 @@
           -- STREAM_LIMIT  |PARTITIONED|
             exchange
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              unnest-map [$$16, $$17, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$21, $$22, 2, $$21, $$22, TRUE, TRUE, TRUE) condition (lt($$c.getField(2), 150)) limit 10
+              unnest-map [$$16, $$17, $$c] <- index-search("LineItem", 0, "test", "LineItem", false, false, 2, $$21, $$22, 2, $$21, $$22, true, true, true) condition (lt($$c.getField(2), 150)) limit 10
               -- BTREE_SEARCH  |PARTITIONED|
                 exchange
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -24,11 +24,11 @@
                       -- STREAM_PROJECT  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          unnest-map [$$20, $$21, $$22] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$19, TRUE, FALSE, FALSE)
+                          unnest-map [$$20, $$21, $$22] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", false, false, 0, 1, $$19, true, false, false)
                           -- BTREE_SEARCH  |PARTITIONED|
                             exchange
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               assign [$$19] <- [150]
                               -- ASSIGN  |PARTITIONED|
                                 empty-tuple-source
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/enforcing_item_type/enforcing_item_type.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/enforcing_item_type/enforcing_item_type.1.adm
index f0da628..01a297a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/enforcing_item_type/enforcing_item_type.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/enforcing_item_type/enforcing_item_type.1.adm
@@ -6,7 +6,7 @@
     -- STREAM_PROJECT  |UNPARTITIONED|
       assign [$$20] <- [{"id": get-item($$t, 0), "v": get-item($$t, 1)}]
       -- ASSIGN  |UNPARTITIONED|
-        unnest $$t <- scan-collection(ordered-list-constructor(ordered-list-constructor(29, cast({ f1: "a", f2: 3 }))))
+        unnest $$t <- scan-collection(ordered-list-constructor(ordered-list-constructor(29, cast({ "f1": "a", "f2": 3 }))))
         -- UNNEST  |UNPARTITIONED|
           empty-tuple-source
           -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.1.adm
index 379603f..c757014 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.1.adm
@@ -2,7 +2,7 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    assign [$$5] <- [{ f1: 5, f2: 6, f3: 7 }]
+    assign [$$5] <- [{ "f1": 5, "f2": 6, "f3": 7 }]
     -- ASSIGN  |UNPARTITIONED|
       empty-tuple-source
       -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.adm
index aafd2d0..7a73900 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.adm
@@ -2,7 +2,7 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    assign [$$14] <- [TRUE]
+    assign [$$14] <- [true]
     -- ASSIGN  |UNPARTITIONED|
       project ([])
       -- STREAM_PROJECT  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.adm
index d2f3187..775cc2a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.adm
@@ -4,7 +4,7 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     project ([$$16])
     -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$16] <- [or(TRUE, lt(get-year(current-date()), $$x))]
+      assign [$$16] <- [or(true, lt(get-year(current-date()), $$x))]
       -- ASSIGN  |UNPARTITIONED|
         exchange
         -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.adm
index e14f391..6faed2f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.13.adm
@@ -10,7 +10,7 @@
         -- 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 [$$16, $$17, $$l] <- index-search("LineItem", 0, "tpch", "LineItem", false, false, 2, $$24, $$25, 2, $$24, $$25, true, true, true)
             -- BTREE_SEARCH  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -22,7 +22,7 @@
                     -- 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 [$$23, $$24, $$25] <- index-search("lineitem_shipdateIx", 0, "tpch", "LineItem", false, false, 1, $$21, 1, $$22, true, true, true)
                         -- BTREE_SEARCH  |PARTITIONED|
                           exchange
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.adm
index e14f391..6faed2f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/rebalance/single_dataset_with_index/single_dataset_with_index.8.adm
@@ -10,7 +10,7 @@
         -- 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 [$$16, $$17, $$l] <- index-search("LineItem", 0, "tpch", "LineItem", false, false, 2, $$24, $$25, 2, $$24, $$25, true, true, true)
             -- BTREE_SEARCH  |PARTITIONED|
               exchange
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -22,7 +22,7 @@
                     -- 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 [$$23, $$24, $$25] <- index-search("lineitem_shipdateIx", 0, "tpch", "LineItem", false, false, 1, $$21, 1, $$22, true, true, true)
                         -- BTREE_SEARCH  |PARTITIONED|
                           exchange
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpcds/q18/q18.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpcds/q18/q18.2.adm
new file mode 100644
index 0000000..6280446
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpcds/q18/q18.2.adm
@@ -0,0 +1 @@
+{ "cnt": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpcds/q18/q18.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpcds/q18/q18.3.adm
new file mode 100644
index 0000000..6280446
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpcds/q18/q18.3.adm
@@ -0,0 +1 @@
+{ "cnt": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index c0784c1..7aef277 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -34,6 +34,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="api">
+      <compilation-unit name="compileonly-2">
+        <output-dir compare="Text">compileonly-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="api">
       <compilation-unit name="multiple-param-values">
         <output-dir compare="Text">multiple-param-values</output-dir>
       </compilation-unit>
@@ -4120,6 +4125,11 @@
         -->
   </test-group>
   <test-group name="ddl">
+    <test-case FilePath="ddl">
+      <compilation-unit name="analyze-dataset-1">
+        <output-dir compare="Text">analyze-dataset-1</output-dir>
+      </compilation-unit>
+    </test-case>
     <test-case FilePath="ddl/create-index">
       <compilation-unit name="create-index-1">
         <output-dir compare="Text">create-index-1</output-dir>
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-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
index 67f8253..f4e241c 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IApplicationContext.java
@@ -31,6 +31,7 @@
 import org.apache.hyracks.api.application.IServiceContext;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.result.IResultSet;
 
 public interface IApplicationContext {
 
@@ -67,6 +68,11 @@
     IHyracksClientConnection getHcc() throws HyracksDataException;
 
     /**
+     * @return a result set provider associated with {@link IHyracksClientConnection}
+     */
+    IResultSet getResultSet() throws HyracksDataException;
+
+    /**
      * @return the cluster coordination service.
      */
     ICoordinationService getCoordinationService();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
index 5538a0a..9599435 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
@@ -103,7 +103,11 @@
         COMPILER_ARRAYINDEX(
                 BOOLEAN,
                 AlgebricksConfig.ARRAY_INDEX_DEFAULT,
-                "Enable/disable using array-indexes in queries");
+                "Enable/disable using array-indexes in queries"),
+        COMPILER_BATCH_LOOKUP(
+                BOOLEAN,
+                AlgebricksConfig.BATCH_LOOKUP_DEFAULT,
+                "Enable/disable batch point-lookups when running queries with secondary indexes");
 
         private final IOptionType type;
         private final Object defaultValue;
@@ -173,6 +177,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 int COMPILER_PARALLELISM_AS_STORAGE = 0;
 
     public CompilerProperties(PropertiesAccessor accessor) {
@@ -246,4 +252,8 @@
     public int getExternalScanMemorySize() {
         return accessor.getInt(Option.COMPILER_EXTERNALSCANMEMORY);
     }
+
+    public boolean isBatchLookup() {
+        return accessor.getBoolean(Option.COMPILER_BATCH_LOOKUP);
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
index b85c0be..478bd46 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
@@ -41,7 +41,8 @@
         SINGLE_PARTITION_NGRAM_INVIX,
         LENGTH_PARTITIONED_WORD_INVIX,
         LENGTH_PARTITIONED_NGRAM_INVIX,
-        ARRAY;
+        ARRAY,
+        SAMPLE;
     }
 
     public enum TransactionState {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
index d498d69..ab3a925 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());
 
         PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
         physOptConf.setFrameSize(frameSize);
@@ -95,6 +97,7 @@
         physOptConf.setMinMemoryAllocation(minMemoryAllocation);
         physOptConf.setArrayIndexEnabled(arrayIndex);
         physOptConf.setExternalScanBufferSize(externalScanBufferSize);
+        physOptConf.setBatchLookup(batchLookup);
         return physOptConf;
     }
 
@@ -117,6 +120,14 @@
                 sourceLoc);
     }
 
+    public static int getGroupByNumFrames(CompilerProperties compilerProperties,
+            Map<String, Object> querySpecificConfig, SourceLocation sourceLoc) throws AlgebricksException {
+        return getFrameLimit(CompilerProperties.COMPILER_GROUPMEMORY_KEY,
+                (String) querySpecificConfig.get(CompilerProperties.COMPILER_GROUPMEMORY_KEY),
+                compilerProperties.getGroupMemorySize(), compilerProperties.getFrameSize(),
+                MIN_FRAME_LIMIT_FOR_GROUP_BY, sourceLoc);
+    }
+
     public static int getTextSearchNumFrames(CompilerProperties compilerProperties,
             Map<String, Object> querySpecificConfig, SourceLocation sourceLoc) throws AlgebricksException {
         return getFrameLimit(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
index 249505b..58d5f9b 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
@@ -72,6 +72,10 @@
         this.partition = partition;
     }
 
+    public IResource getResource() {
+        return resource;
+    }
+
     @Override
     public IIndex createInstance(INCServiceContext ncServiceCtx) throws HyracksDataException {
         return resource.createInstance(ncServiceCtx);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index fc234ef..2235005 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -265,6 +265,7 @@
     S3_REGION_NOT_SUPPORTED(1170),
     COMPILATION_SET_OPERATION_ERROR(1171),
     INVALID_TIMEZONE(1172),
+    INVALID_PARAM_VALUE_ALLOWED_VALUE(1173),
 
     // Feed errors
     DATAFLOW_ILLEGAL_STATE(3001),
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/NoOpWarningCollector.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/NoOpWarningCollector.java
index a036b7e..6ac805b 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/NoOpWarningCollector.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/NoOpWarningCollector.java
@@ -20,27 +20,10 @@
 package org.apache.asterix.common.exceptions;
 
 import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.Warning;
 
-public final class NoOpWarningCollector implements IWarningCollector {
-
-    public static final IWarningCollector INSTANCE = new NoOpWarningCollector();
+public final class NoOpWarningCollector {
+    public static final IWarningCollector INSTANCE = org.apache.hyracks.api.exceptions.NoOpWarningCollector.INSTANCE;
 
     private NoOpWarningCollector() {
     }
-
-    @Override
-    public void warn(Warning warning) {
-        // no-op
-    }
-
-    @Override
-    public boolean shouldWarn() {
-        return false;
-    }
-
-    @Override
-    public long getTotalWarningsCount() {
-        return 0;
-    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
index 28eb553..684ddae 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
@@ -52,6 +52,9 @@
     void refreshDatasetBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
             String datasetName) throws AlgebricksException;
 
+    void analyzeDatasetBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
+            String datasetName) throws AlgebricksException;
+
     void compactBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName, String datasetName)
             throws AlgebricksException;
 
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java
index 232c8dd..c1c6f18 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/JobUtils.java
@@ -20,11 +20,14 @@
 package org.apache.asterix.common.utils;
 
 import java.util.EnumSet;
+import java.util.List;
 
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 
 public class JobUtils {
 
@@ -53,4 +56,21 @@
         }
         return jobId;
     }
+
+    public static Pair<JobId, List<IOperatorStats>> runJob(IHyracksClientConnection hcc, JobSpecification spec,
+            EnumSet<JobFlag> jobFlags, boolean waitForCompletion, List<String> statOperatorNames) throws Exception {
+        spec.setMaxReattempts(0);
+        final JobId jobId = hcc.startJob(spec, jobFlags);
+        List<IOperatorStats> opStats = null;
+        if (waitForCompletion) {
+            String nameBefore = Thread.currentThread().getName();
+            try {
+                Thread.currentThread().setName(nameBefore + " : WaitForCompletionForJobId: " + jobId);
+                opStats = hcc.waitForCompletion(jobId, statOperatorNames);
+            } finally {
+                Thread.currentThread().setName(nameBefore);
+            }
+        }
+        return new Pair<>(jobId, opStats);
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index b98dc43..7455a3b 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -267,6 +267,7 @@
 1170 = Provided S3 region is not supported: '%1$s'
 1171 = Unable to process %1$s clause. %2$s
 1172 = Provided timezone is invalid: '%1$s'
+1173 = Invalid value for parameter '%1$s', allowed value(s): %2$s
 
 # Feed Errors
 3001 = Illegal state.
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss
index 94311f8..68c12f4 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss
@@ -46,6 +46,7 @@
 .options {
     display: flex;
     flex-flow: row;
+    overflow: auto;
 }
 
 .codemirror-container {
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
index f885edb..cb46a6a 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
@@ -538,7 +538,7 @@
         this.queryString = '';
         this.selected = 'Default';
       } else {
-        this.queryString = 'USE ' + this.selected + '; \n';
+        this.queryString = 'USE ' + this.selected + '; \n' + this.queryString;
       }
       this.editor.getDoc().setValue(this.queryString);
       this.editor.execCommand('goDocEnd')
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html
index a131353..4b6ecc5 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html
@@ -97,7 +97,7 @@
         [links]="edgesArr"
         [nodes]="nodesArr"
         [draggingEnabled]="false"
-        [zoomSpeed]="0.025"
+        [zoomSpeed]="0.015"
         [update$]="update$"
         [layoutSettings]="{
                 orientation: planOrientation,
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss
index 6c40a68..9601675 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss
@@ -36,7 +36,6 @@
 }
 
 .panel {
-  order: 2;
   display: flex;
   flex-flow: column;
   justify-content: stretch;
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
index 0fbf6f6..c5754762 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
@@ -30,7 +30,8 @@
                     <button *ngIf="isCSV" mat-button class='button export' (click)='openJSONExportPicker()' matTooltip="Export CSV file to Computer">EXPORT</button>
                     <button *ngIf="isCSV == false" mat-button class='button export' (click)='openJSONExportPicker()' matTooltip="Export JSON/JSONL file to Computer">EXPORT</button>
                 </span>
-                <mat-paginator *ngIf="this.planVisible === false" [showFirstLastButtons]="true" [length]='metrics.resultCount' [pageSize]='pagedefaults.pageSize' [pageSizeOptions]='pageSizeOptions' (page)='showResults($event, false)'>
+
+                <mat-paginator *ngIf="this.planVisible === false" [showFirstLastButtons]="true" [length]='metrics.resultCount' [pageSize]='pageSize' [pageSizeOptions]='pageSizeOptions' (page)='showResults($event, false)'>
                 </mat-paginator>
             </div>
             <div *ngIf='treeVisible' class='navi-data' class='navi-data'>
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
index 5309991..66a6dd6 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
@@ -60,7 +60,8 @@
     currentRange: any;
     /* see 10 records as initial set */
     pagedefaults: any = { pageIndex: 0, pageSize:10, lenght: 0};
-    pageSizeOptions = [5, 10, 25, 100, 200];
+    pageSize = 10;
+    pageSizeOptions = [5, 10, 25, 100, 200, 300, 400];
     viewMode = 'JSON';
     showGoTop = false;
     showGoBottom = false;
@@ -141,6 +142,9 @@
     }
 
     showResults(range, expanded) {
+        // update pagesize
+        this.pageSize = range.pageSize;
+
         this.currentRange = range;
         this.currentIndex = this.currentRange.pageIndex;
 
diff --git a/asterixdb/asterix-external-data/pom.xml b/asterixdb/asterix-external-data/pom.xml
index dbb2382..abc8c6c 100644
--- a/asterixdb/asterix-external-data/pom.xml
+++ b/asterixdb/asterix-external-data/pom.xml
@@ -518,6 +518,10 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>com.google.cloud.bigdataoss</groupId>
+      <artifactId>gcs-connector</artifactId>
+    </dependency>
     <!-- apply patch for HADOOP-17225 to workaround CVE-2019-10172 -->
     <dependency>
       <groupId>org.codehaus.jackson</groupId>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/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 803e657..ff93a46 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
@@ -28,6 +28,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;
@@ -52,7 +54,7 @@
         //Configure Hadoop S3 input splits
         JobConf conf = createHdfsConf(serviceCtx, configuration);
         int numberOfPartitions = getPartitionConstraint().getLocations().length;
-        ExternalDataUtils.AwsS3.configureAwsS3HdfsJobConf(conf, configuration, numberOfPartitions);
+        S3Utils.configureAwsS3HdfsJobConf(conf, configuration, numberOfPartitions);
         configureHdfsConf(conf, configuration);
     }
 
@@ -89,8 +91,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()) {
@@ -105,7 +106,7 @@
     }
 
     private static void appendFileURI(StringBuilder builder, String container, S3Object file) {
-        builder.append(ExternalDataConstants.AwsS3.HADOOP_S3_PROTOCOL);
+        builder.append(S3Constants.HADOOP_S3_PROTOCOL);
         builder.append("://");
         builder.append(container);
         builder.append('/');
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java
index cdb3834..bbfece2 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.external.input.record.reader.azure.blob;
 
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureBlobClient;
 import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
 
 import java.io.IOException;
@@ -31,7 +32,6 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
 import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.util.LogRedactionUtil;
 
@@ -86,7 +86,7 @@
     private BlobServiceClient buildAzureClient(IApplicationContext appCtx, Map<String, String> configuration)
             throws HyracksDataException {
         try {
-            return ExternalDataUtils.Azure.buildAzureBlobClient(appCtx, configuration);
+            return buildAzureBlobClient(appCtx, configuration);
         } catch (CompilationException ex) {
             throw HyracksDataException.create(ex);
         }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java
index 064b319..55c0521 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.external.input.record.reader.azure.blob;
 
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureBlobClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listBlobItems;
+
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
@@ -57,9 +60,9 @@
         // Ensure the validity of include/exclude
         ExternalDataUtils.validateIncludeExclude(configuration);
         IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
-        BlobServiceClient blobServiceClient = ExternalDataUtils.Azure.buildAzureBlobClient(appCtx, configuration);
-        List<BlobItem> filesOnly = ExternalDataUtils.Azure.listBlobItems(blobServiceClient, configuration,
-                includeExcludeMatcher, warningCollector);
+        BlobServiceClient blobServiceClient = buildAzureBlobClient(appCtx, configuration);
+        List<BlobItem> filesOnly =
+                listBlobItems(blobServiceClient, configuration, includeExcludeMatcher, warningCollector);
 
         // Distribute work load amongst the partitions
         distributeWorkLoad(filesOnly, getPartitionsCount());
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java
index e34d188..7a95222 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.external.input.record.reader.azure.datalake;
 
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureDatalakeClient;
 import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
 
 import java.io.IOException;
@@ -31,7 +32,6 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
 import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.util.LogRedactionUtil;
 
@@ -86,7 +86,7 @@
     private DataLakeServiceClient buildAzureClient(IApplicationContext appCtx, Map<String, String> configuration)
             throws HyracksDataException {
         try {
-            return ExternalDataUtils.Azure.buildAzureDatalakeClient(appCtx, configuration);
+            return buildAzureDatalakeClient(appCtx, configuration);
         } catch (CompilationException ex) {
             throw HyracksDataException.create(ex);
         }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java
index e9f8d4c..929cb6e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.external.input.record.reader.azure.datalake;
 
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureDatalakeClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listDatalakePathItems;
+
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
@@ -57,9 +60,9 @@
         // Ensure the validity of include/exclude
         ExternalDataUtils.validateIncludeExclude(configuration);
         IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
-        DataLakeServiceClient client = ExternalDataUtils.Azure.buildAzureDatalakeClient(appCtx, configuration);
-        List<PathItem> filesOnly = ExternalDataUtils.Azure.listDatalakePathItems(client, configuration,
-                includeExcludeMatcher, warningCollector);
+        DataLakeServiceClient client = buildAzureDatalakeClient(appCtx, configuration);
+        List<PathItem> filesOnly =
+                listDatalakePathItems(client, configuration, includeExcludeMatcher, warningCollector);
 
         // Distribute work load amongst the partitions
         distributeWorkLoad(filesOnly, getPartitionsCount());
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java
index c2251df..e08013c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java
@@ -18,6 +18,11 @@
  */
 package org.apache.asterix.external.input.record.reader.azure.parquet;
 
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_BLOB_PROTOCOL;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureBlobClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.configureAzureHdfsJobConf;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listBlobItems;
+
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -47,7 +52,7 @@
     public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
             IWarningCollector warningCollector) throws AlgebricksException, HyracksDataException {
         IApplicationContext appCtx = (IApplicationContext) serviceCtx.getApplicationContext();
-        BlobServiceClient blobServiceClient = ExternalDataUtils.Azure.buildAzureBlobClient(appCtx, configuration);
+        BlobServiceClient blobServiceClient = buildAzureBlobClient(appCtx, configuration);
         //Get endpoint
         String endPoint = extractEndPoint(blobServiceClient.getAccountUrl());
         //Get path
@@ -57,7 +62,7 @@
 
         //Configure Hadoop Azure input splits
         JobConf conf = createHdfsConf(serviceCtx, configuration);
-        ExternalDataUtils.Azure.configureAzureHdfsJobConf(conf, configuration, endPoint);
+        configureAzureHdfsJobConf(conf, configuration, endPoint);
         configureHdfsConf(conf, configuration);
     }
 
@@ -94,8 +99,8 @@
     private static String buildPathURIs(Map<String, String> configuration, IWarningCollector warningCollector,
             BlobServiceClient blobServiceClient, String endPoint) throws CompilationException {
         IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
-        List<BlobItem> filesOnly = ExternalDataUtils.Azure.listBlobItems(blobServiceClient, configuration,
-                includeExcludeMatcher, warningCollector);
+        List<BlobItem> filesOnly =
+                listBlobItems(blobServiceClient, configuration, includeExcludeMatcher, warningCollector);
 
         StringBuilder builder = new StringBuilder();
         String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
@@ -118,7 +123,7 @@
     }
 
     private static void appendFileURI(StringBuilder builder, String container, String endPoint, BlobItem file) {
-        builder.append(ExternalDataConstants.Azure.HADOOP_AZURE_BLOB_PROTOCOL);
+        builder.append(HADOOP_AZURE_BLOB_PROTOCOL);
         builder.append("://");
         builder.append(container);
         builder.append('@');
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java
index db87868..c98fc8b 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java
@@ -18,6 +18,11 @@
  */
 package org.apache.asterix.external.input.record.reader.azure.parquet;
 
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_DATALAKE_PROTOCOL;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.buildAzureDatalakeClient;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.configureAzureHdfsJobConf;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.listDatalakePathItems;
+
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -47,8 +52,7 @@
     public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
             IWarningCollector warningCollector) throws AlgebricksException, HyracksDataException {
         IApplicationContext appCtx = (IApplicationContext) serviceCtx.getApplicationContext();
-        DataLakeServiceClient dataLakeServiceClient =
-                ExternalDataUtils.Azure.buildAzureDatalakeClient(appCtx, configuration);
+        DataLakeServiceClient dataLakeServiceClient = buildAzureDatalakeClient(appCtx, configuration);
 
         //Get endpoint
         String endPoint = extractEndPoint(dataLakeServiceClient.getAccountUrl());
@@ -61,7 +65,7 @@
 
         //Configure Hadoop Azure input splits
         JobConf conf = createHdfsConf(serviceCtx, configuration);
-        ExternalDataUtils.Azure.configureAzureHdfsJobConf(conf, configuration, endPoint);
+        configureAzureHdfsJobConf(conf, configuration, endPoint);
         configureHdfsConf(conf, configuration);
     }
 
@@ -98,8 +102,8 @@
     private static String buildPathURIs(Map<String, String> configuration, IWarningCollector warningCollector,
             DataLakeServiceClient dataLakeServiceClient, String endPoint) throws CompilationException {
         IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
-        List<PathItem> filesOnly = ExternalDataUtils.Azure.listDatalakePathItems(dataLakeServiceClient, configuration,
-                includeExcludeMatcher, warningCollector);
+        List<PathItem> filesOnly =
+                listDatalakePathItems(dataLakeServiceClient, configuration, includeExcludeMatcher, warningCollector);
 
         StringBuilder builder = new StringBuilder();
         String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
@@ -122,7 +126,7 @@
     }
 
     private static void appendFileURI(StringBuilder builder, String container, String endPoint, PathItem file) {
-        builder.append(ExternalDataConstants.Azure.HADOOP_AZURE_DATALAKE_PROTOCOL);
+        builder.append(HADOOP_AZURE_DATALAKE_PROTOCOL);
         builder.append("://");
         builder.append(container);
         builder.append('@');
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
index 652fa3e..007e8be 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
@@ -32,7 +32,7 @@
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
 import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.google.gcs.GCSUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.util.CleanupUtils;
@@ -134,7 +134,7 @@
 
     private Storage buildClient(Map<String, String> configuration) throws HyracksDataException {
         try {
-            return ExternalDataUtils.GCS.buildClient(configuration);
+            return GCSUtils.buildClient(configuration);
         } catch (CompilationException ex) {
             throw HyracksDataException.create(ex);
         }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java
index 0e7ea90..278c1ad 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java
@@ -19,33 +19,23 @@
 package org.apache.asterix.external.input.record.reader.gcs;
 
 import static org.apache.asterix.external.util.ExternalDataUtils.getIncludeExcludeMatchers;
-import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
 
-import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
 import java.util.PriorityQueue;
-import java.util.function.BiPredicate;
-import java.util.regex.Matcher;
 
-import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.api.AsterixInputStream;
 import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
-import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.google.gcs.GCSUtils;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.IServiceContext;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.Warning;
 
-import com.google.api.gax.paging.Page;
-import com.google.cloud.BaseServiceException;
 import com.google.cloud.storage.Blob;
-import com.google.cloud.storage.Storage;
 
 public class GCSInputStreamFactory extends AbstractExternalInputStreamFactory {
 
@@ -63,57 +53,16 @@
 
         // Ensure the validity of include/exclude
         ExternalDataUtils.validateIncludeExclude(configuration);
-
-        // Prepare to retrieve the objects
-        List<Blob> filesOnly = new ArrayList<>();
-        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-        Storage gcs = ExternalDataUtils.GCS.buildClient(configuration);
-        Storage.BlobListOption options = Storage.BlobListOption.prefix(ExternalDataUtils.getPrefix(configuration));
-        Page<Blob> items;
-
-        try {
-            items = gcs.list(container, options);
-        } catch (BaseServiceException ex) {
-            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-        }
-
-        // Collect the paths to files only
         IncludeExcludeMatcher includeExcludeMatcher = getIncludeExcludeMatchers(configuration);
-        collectAndFilterFiles(items, includeExcludeMatcher.getPredicate(), includeExcludeMatcher.getMatchersList(),
-                filesOnly);
 
-        // Warn if no files are returned
-        if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
-            Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
-            warningCollector.warn(warning);
-        }
+        // get the items
+        List<Blob> filesOnly = GCSUtils.listItems(configuration, includeExcludeMatcher, warningCollector);
 
         // Distribute work load amongst the partitions
         distributeWorkLoad(filesOnly, getPartitionsCount());
     }
 
     /**
-     * AWS S3 returns all the objects as paths, not differentiating between folder and files. The path is considered
-     * a file if it does not end up with a "/" which is the separator in a folder structure.
-     *
-     * @param items List of returned objects
-     */
-    private void collectAndFilterFiles(Page<Blob> items, BiPredicate<List<Matcher>, String> predicate,
-            List<Matcher> matchers, List<Blob> filesOnly) {
-        for (Blob item : items.iterateAll()) {
-            // skip folders
-            if (item.getName().endsWith("/")) {
-                continue;
-            }
-
-            // No filter, add file
-            if (predicate.test(matchers, item.getName())) {
-                filesOnly.add(item);
-            }
-        }
-    }
-
-    /**
      * To efficiently utilize the parallelism, work load will be distributed amongst the partitions based on the file
      * size.
      *
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/parquet/GCSParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/parquet/GCSParquetReaderFactory.java
new file mode 100644
index 0000000..2887415
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/parquet/GCSParquetReaderFactory.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.gcs.parquet;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.external.input.HDFSDataSourceFactory;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.google.gcs.GCSConstants;
+import org.apache.asterix.external.util.google.gcs.GCSUtils;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+
+import com.google.cloud.storage.Blob;
+
+public class GCSParquetReaderFactory extends HDFSDataSourceFactory {
+    private static final long serialVersionUID = -6140824803254158253L;
+    private static final List<String> recordReaderNames =
+            Collections.singletonList(ExternalDataConstants.KEY_ADAPTER_NAME_GCS);
+
+    @Override
+    public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
+            IWarningCollector warningCollector) throws AlgebricksException, HyracksDataException {
+        // get path
+        String path = buildPathURIs(configuration, warningCollector);
+
+        // put GCS configurations to AsterixDB's Hadoop configuration
+        putGCSConfToHadoopConf(configuration, path);
+
+        // configure hadoop input splits
+        JobConf conf = createHdfsConf(serviceCtx, configuration);
+        int numberOfPartitions = getPartitionConstraint().getLocations().length;
+        GCSUtils.configureHdfsJobConf(conf, configuration, numberOfPartitions);
+        configureHdfsConf(conf, configuration);
+    }
+
+    @Override
+    public List<String> getRecordReaderNames() {
+        return recordReaderNames;
+    }
+
+    @Override
+    public Set<String> getReaderSupportedFormats() {
+        return Collections.singleton(ExternalDataConstants.FORMAT_PARQUET);
+    }
+
+    /**
+     * Prepare Hadoop configurations to read parquet files
+     *
+     * @param path Comma-delimited paths
+     */
+    private static void putGCSConfToHadoopConf(Map<String, String> configuration, String path) {
+        configuration.put(ExternalDataConstants.KEY_PATH, path);
+        configuration.put(ExternalDataConstants.KEY_INPUT_FORMAT, ExternalDataConstants.INPUT_FORMAT_PARQUET);
+        configuration.put(ExternalDataConstants.KEY_PARSER, ExternalDataConstants.FORMAT_NOOP);
+    }
+
+    /**
+     * Build Google Cloud Storage path-style for the requested files
+     *
+     * @param configuration    properties
+     * @param warningCollector warning collector
+     * @return Comma-delimited paths (e.g., "gs://bucket/file1.parquet,gs://bucket/file2.parquet")
+     * @throws CompilationException Compilation exception
+     */
+    private static String buildPathURIs(Map<String, String> configuration, IWarningCollector warningCollector)
+            throws CompilationException {
+        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+        // Ensure the validity of include/exclude
+        ExternalDataUtils.validateIncludeExclude(configuration);
+        IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
+
+        // get the items
+        List<Blob> filesOnly = GCSUtils.listItems(configuration, includeExcludeMatcher, warningCollector);
+        StringBuilder builder = new StringBuilder();
+
+        if (!filesOnly.isEmpty()) {
+            appendFileURI(builder, container, filesOnly.get(0));
+            for (int i = 1; i < filesOnly.size(); i++) {
+                builder.append(',');
+                appendFileURI(builder, container, filesOnly.get(i));
+            }
+        }
+
+        return builder.toString();
+    }
+
+    private static void appendFileURI(StringBuilder builder, String container, Blob file) {
+        builder.append(GCSConstants.HADOOP_GCS_PROTOCOL);
+        builder.append("://");
+        builder.append(container);
+        builder.append('/');
+        builder.append(file.getName());
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/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/util/ExternalDataConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
index f0b9c90..429706e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
@@ -333,107 +333,4 @@
          */
         public static final Set<String> VALID_TIME_ZONES = Set.of(TimeZone.getAvailableIDs());
     }
-
-    public static class AwsS3 {
-        private AwsS3() {
-            throw new AssertionError("do not instantiate");
-        }
-
-        public static final String REGION_FIELD_NAME = "region";
-        public static final String ACCESS_KEY_ID_FIELD_NAME = "accessKeyId";
-        public static final String SECRET_ACCESS_KEY_FIELD_NAME = "secretAccessKey";
-        public static final String SESSION_TOKEN_FIELD_NAME = "sessionToken";
-        public static final String SERVICE_END_POINT_FIELD_NAME = "serviceEndpoint";
-
-        // AWS S3 specific error codes
-        public static final String ERROR_INTERNAL_ERROR = "InternalError";
-        public static final String ERROR_SLOW_DOWN = "SlowDown";
-        public static final String ERROR_METHOD_NOT_IMPLEMENTED = "NotImplemented";
-
-        public static boolean isRetryableError(String errorCode) {
-            return errorCode.equals(ERROR_INTERNAL_ERROR) || errorCode.equals(ERROR_SLOW_DOWN);
-        }
-
-        /*
-         * Hadoop-AWS
-         * AWS connectors for s3 and s3n are deprecated.
-         */
-        public static final String HADOOP_ACCESS_KEY_ID = "fs.s3a.access.key";
-        public static final String HADOOP_SECRET_ACCESS_KEY = "fs.s3a.secret.key";
-        public static final String HADOOP_SESSION_TOKEN = "fs.s3a.session.token";
-        public static final String HADOOP_REGION = "fs.s3a.region";
-        public static final String HADOOP_SERVICE_END_POINT = "fs.s3a.endpoint";
-
-        /*
-         * Internal configurations
-         */
-        //Allows accessing directories as file system path
-        public static final String HADOOP_PATH_STYLE_ACCESS = "fs.s3a.path.style.access";
-        //The number of maximum HTTP connections in connection pool
-        public static final String HADOOP_S3_CONNECTION_POOL_SIZE = "fs.s3a.connection.maximum";
-        //S3 used protocol
-        public static final String HADOOP_S3_PROTOCOL = "s3a";
-
-        //Hadoop credentials provider key
-        public static final String HADOOP_CREDENTIAL_PROVIDER_KEY = "fs.s3a.aws.credentials.provider";
-        //Anonymous credential provider
-        public static final String HADOOP_ANONYMOUS_ACCESS = "org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider";
-        //Temporary credential provider
-        public static final String HADOOP_TEMP_ACCESS = "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
-
-    }
-
-    /*
-     * Note: Azure Blob and Azure Datalake use identical authentication, so they are using the same properties.
-     * If they end up diverging, then properties for AzureBlob and AzureDataLake need to be created.
-     */
-    public static class Azure {
-        private Azure() {
-            throw new AssertionError("do not instantiate");
-        }
-
-        /*
-         * Asterix Configuration Keys
-         */
-        public static final String MANAGED_IDENTITY_ID_FIELD_NAME = "managedIdentityId";
-        public static final String ACCOUNT_NAME_FIELD_NAME = "accountName";
-        public static final String ACCOUNT_KEY_FIELD_NAME = "accountKey";
-        public static final String SHARED_ACCESS_SIGNATURE_FIELD_NAME = "sharedAccessSignature";
-        public static final String TENANT_ID_FIELD_NAME = "tenantId";
-        public static final String CLIENT_ID_FIELD_NAME = "clientId";
-        public static final String CLIENT_SECRET_FIELD_NAME = "clientSecret";
-        public static final String CLIENT_CERTIFICATE_FIELD_NAME = "clientCertificate";
-        public static final String CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME = "clientCertificatePassword";
-        public static final String ENDPOINT_FIELD_NAME = "endpoint";
-
-        // Specific Azure data lake property
-        /*
-        The behavior of Data Lake (true file system) is to read the files of the specified prefix only, example:
-        storage/myData/personal/file1.json
-        storage/myData/personal/file2.json
-        storage/myData/file3.json
-        If the prefix used is "myData", then only the file file3.json is read. However, if the property "recursive"
-        is set to "true" when creating the external dataset, then it goes recursively overall the paths, and the result
-        is file1.json, file2.json and file3.json.
-         */
-        public static final String RECURSIVE_FIELD_NAME = "recursive";
-
-        /*
-         * Hadoop-Azure
-         */
-        //Used when accountName and accessKey are provided
-        public static final String HADOOP_AZURE_FS_ACCOUNT_KEY = "fs.azure.account.key";
-        //Used when a connectionString is provided
-        public static final String HADOOP_AZURE_FS_SAS = "fs.azure.sas";
-        public static final String HADOOP_AZURE_BLOB_PROTOCOL = "wasbs";
-        public static final String HADOOP_AZURE_DATALAKE_PROTOCOL = "abfss";
-    }
-
-    public static class GCS {
-        private GCS() {
-            throw new AssertionError("do not instantiate");
-        }
-
-        public static final String JSON_CREDENTIALS_FIELD_NAME = "jsonCredentials";
-    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
index 8e38eed..62dc074 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
@@ -18,43 +18,9 @@
  */
 package org.apache.asterix.external.util;
 
-import static com.google.cloud.storage.Storage.BlobListOption;
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_SOURCE_ERROR;
 import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_REQ_PARAM_VAL;
 import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_NOT_ALLOWED_AT_SAME_TIME;
 import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_REQUIRED;
-import static org.apache.asterix.common.exceptions.ErrorCode.PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT;
-import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
-import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT;
-import static org.apache.asterix.common.exceptions.ErrorCode.S3_REGION_NOT_SUPPORTED;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ACCESS_KEY_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ERROR_METHOD_NOT_IMPLEMENTED;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_ACCESS_KEY_ID;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_ANONYMOUS_ACCESS;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_CREDENTIAL_PROVIDER_KEY;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_PATH_STYLE_ACCESS;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_S3_CONNECTION_POOL_SIZE;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_S3_PROTOCOL;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_SECRET_ACCESS_KEY;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_SESSION_TOKEN;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_TEMP_ACCESS;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ACCOUNT_KEY_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ACCOUNT_NAME_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_CERTIFICATE_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_SECRET_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ENDPOINT_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_BLOB_PROTOCOL;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_FS_ACCOUNT_KEY;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_FS_SAS;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.MANAGED_IDENTITY_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.RECURSIVE_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.SHARED_ACCESS_SIGNATURE_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.Azure.TENANT_ID_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.GCS.JSON_CREDENTIALS_FIELD_NAME;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ADAPTER_NAME_GCS;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_DELIMITER;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ESCAPE;
@@ -65,18 +31,14 @@
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_QUOTE;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_END;
 import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_START;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.validateAzureBlobProperties;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureUtils.validateAzureDataLakeProperties;
+import static org.apache.asterix.external.util.google.gcs.GCSUtils.validateProperties;
 import static org.apache.asterix.runtime.evaluators.functions.StringEvaluatorUtils.RESERVED_REGEX_CHARS;
-import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
 
-import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.net.URI;
-import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Base64;
@@ -85,7 +47,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;
@@ -106,20 +67,17 @@
 import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
 import org.apache.asterix.external.library.JavaLibrary;
 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.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.dataflow.common.data.parsers.BooleanParserFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.FloatParserFactory;
@@ -129,46 +87,6 @@
 import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 import org.apache.hyracks.util.StorageUtil;
 
-import com.azure.core.credential.AzureSasCredential;
-import com.azure.core.http.rest.PagedIterable;
-import com.azure.identity.ClientCertificateCredentialBuilder;
-import com.azure.identity.ClientSecretCredentialBuilder;
-import com.azure.identity.ManagedIdentityCredentialBuilder;
-import com.azure.storage.blob.BlobContainerClient;
-import com.azure.storage.blob.BlobServiceClient;
-import com.azure.storage.blob.BlobServiceClientBuilder;
-import com.azure.storage.blob.models.BlobItem;
-import com.azure.storage.blob.models.ListBlobsOptions;
-import com.azure.storage.common.StorageSharedKeyCredential;
-import com.azure.storage.common.policy.RequestRetryOptions;
-import com.azure.storage.file.datalake.DataLakeFileSystemClient;
-import com.azure.storage.file.datalake.DataLakeServiceClient;
-import com.azure.storage.file.datalake.DataLakeServiceClientBuilder;
-import com.azure.storage.file.datalake.models.ListPathsOptions;
-import com.azure.storage.file.datalake.models.PathItem;
-import com.google.api.gax.paging.Page;
-import com.google.auth.oauth2.ServiceAccountCredentials;
-import com.google.cloud.storage.Blob;
-import com.google.cloud.storage.Storage;
-import com.google.cloud.storage.StorageOptions;
-
-import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
-import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
-import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
-import software.amazon.awssdk.core.exception.SdkException;
-import software.amazon.awssdk.regions.Region;
-import software.amazon.awssdk.services.s3.S3Client;
-import software.amazon.awssdk.services.s3.S3ClientBuilder;
-import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
-import software.amazon.awssdk.services.s3.model.ListObjectsResponse;
-import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
-import software.amazon.awssdk.services.s3.model.ListObjectsV2Response;
-import software.amazon.awssdk.services.s3.model.S3Exception;
-import software.amazon.awssdk.services.s3.model.S3Object;
-import software.amazon.awssdk.services.s3.model.S3Response;
-
 public class ExternalDataUtils {
     private static final Map<ATypeTag, IValueParserFactory> valueParserFactoryMap = new EnumMap<>(ATypeTag.class);
     private static final int DEFAULT_MAX_ARGUMENT_SZ = 1024 * 1024;
@@ -604,16 +522,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
@@ -844,7 +762,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)
@@ -893,1008 +811,6 @@
         return encoder.encodeToString(byteArrayOutputStream.toByteArray());
     }
 
-    public static class AwsS3 {
-        private AwsS3() {
-            throw new AssertionError("do not instantiate");
-        }
-
-        /**
-         * Builds the S3 client using the provided configuration
-         *
-         * @param configuration properties
-         * @return S3 client
-         * @throws CompilationException CompilationException
-         */
-        public static S3Client buildAwsS3Client(Map<String, String> configuration) throws CompilationException {
-            // TODO(Hussain): Need to ensure that all required parameters are present in a previous step
-            String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
-            String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
-            String sessionToken = configuration.get(ExternalDataConstants.AwsS3.SESSION_TOKEN_FIELD_NAME);
-            String regionId = configuration.get(ExternalDataConstants.AwsS3.REGION_FIELD_NAME);
-            String serviceEndpoint = configuration.get(ExternalDataConstants.AwsS3.SERVICE_END_POINT_FIELD_NAME);
-
-            S3ClientBuilder builder = S3Client.builder();
-
-            // Credentials
-            AwsCredentialsProvider credentialsProvider;
-
-            // No auth required
-            if (accessKeyId == null) {
-                credentialsProvider = AnonymousCredentialsProvider.create();
-            } else {
-                // auth required, check for temporary or permanent credentials
-                if (sessionToken != null) {
-                    credentialsProvider = StaticCredentialsProvider
-                            .create(AwsSessionCredentials.create(accessKeyId, secretAccessKey, sessionToken));
-                } else {
-                    credentialsProvider =
-                            StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKeyId, secretAccessKey));
-                }
-            }
-
-            builder.credentialsProvider(credentialsProvider);
-
-            // Validate the region
-            List<Region> regions = S3Client.serviceMetadata().regions();
-            Optional<Region> selectedRegion =
-                    regions.stream().filter(region -> region.id().equals(regionId)).findFirst();
-
-            if (selectedRegion.isEmpty()) {
-                throw new CompilationException(S3_REGION_NOT_SUPPORTED, regionId);
-            }
-            builder.region(selectedRegion.get());
-
-            // Validate the service endpoint if present
-            if (serviceEndpoint != null) {
-                try {
-                    URI uri = new URI(serviceEndpoint);
-                    try {
-                        builder.endpointOverride(uri);
-                    } catch (NullPointerException ex) {
-                        throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-                    }
-                } catch (URISyntaxException ex) {
-                    throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR,
-                            String.format("Invalid service endpoint %s", serviceEndpoint));
-                }
-            }
-
-            return builder.build();
-        }
-
-        /**
-         * Builds the S3 client using the provided configuration
-         *
-         * @param configuration      properties
-         * @param numberOfPartitions number of partitions in the cluster
-         */
-        public static void configureAwsS3HdfsJobConf(JobConf conf, Map<String, String> configuration,
-                int numberOfPartitions) {
-            String accessKeyId = configuration.get(ExternalDataConstants.AwsS3.ACCESS_KEY_ID_FIELD_NAME);
-            String secretAccessKey = configuration.get(ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME);
-            String sessionToken = configuration.get(ExternalDataConstants.AwsS3.SESSION_TOKEN_FIELD_NAME);
-            String serviceEndpoint = configuration.get(ExternalDataConstants.AwsS3.SERVICE_END_POINT_FIELD_NAME);
-
-            //Disable caching S3 FileSystem
-            HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_S3_PROTOCOL);
-
-            /*
-             * Authentication Methods:
-             * 1- Anonymous: no accessKeyId and no secretAccessKey
-             * 2- Temporary: has to provide accessKeyId, secretAccessKey and sessionToken
-             * 3- Private: has to provide accessKeyId and secretAccessKey
-             */
-            if (accessKeyId == null) {
-                //Tells hadoop-aws it is an anonymous access
-                conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_ANONYMOUS_ACCESS);
-            } else {
-                conf.set(HADOOP_ACCESS_KEY_ID, accessKeyId);
-                conf.set(HADOOP_SECRET_ACCESS_KEY, secretAccessKey);
-                if (sessionToken != null) {
-                    conf.set(HADOOP_SESSION_TOKEN, sessionToken);
-                    //Tells hadoop-aws it is a temporary access
-                    conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_TEMP_ACCESS);
-                }
-            }
-
-            /*
-             * This is to allow S3 definition to have path-style form. Should always be true to match the current
-             * way we access files in S3
-             */
-            conf.set(HADOOP_PATH_STYLE_ACCESS, ExternalDataConstants.TRUE);
-
-            /*
-             * Set the size of S3 connection pool to be the number of partitions
-             */
-            conf.set(HADOOP_S3_CONNECTION_POOL_SIZE, String.valueOf(numberOfPartitions));
-
-            if (serviceEndpoint != null) {
-                // Validation of the URL should be done at hadoop-aws level
-                conf.set(ExternalDataConstants.AwsS3.HADOOP_SERVICE_END_POINT, serviceEndpoint);
-            } else {
-                //Region is ignored and buckets could be found by the central endpoint
-                conf.set(ExternalDataConstants.AwsS3.HADOOP_SERVICE_END_POINT, Constants.CENTRAL_ENDPOINT);
-            }
-        }
-
-        /**
-         * Validate external dataset properties
-         *
-         * @param configuration properties
-         * @throws CompilationException Compilation exception
-         */
-        public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
-                IWarningCollector collector) throws CompilationException {
-
-            // check if the format property is present
-            if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
-                throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
-            }
-
-            // Both parameters should be passed, or neither should be passed (for anonymous/no auth)
-            String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
-            String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
-            if (accessKeyId == null || secretAccessKey == null) {
-                // If one is passed, the other is required
-                if (accessKeyId != null) {
-                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
-                            ACCESS_KEY_ID_FIELD_NAME);
-                } else if (secretAccessKey != null) {
-                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
-                            SECRET_ACCESS_KEY_FIELD_NAME);
-                }
-            }
-
-            validateIncludeExclude(configuration);
-
-            // Check if the bucket is present
-            S3Client s3Client = buildAwsS3Client(configuration);
-            S3Response response;
-            boolean useOldApi = false;
-            String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-            String prefix = getPrefix(configuration);
-
-            try {
-                response = isBucketEmpty(s3Client, container, prefix, false);
-            } catch (S3Exception ex) {
-                // Method not implemented, try falling back to old API
-                try {
-                    // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
-                    if (ex.awsErrorDetails().errorCode().equals(ERROR_METHOD_NOT_IMPLEMENTED)) {
-                        useOldApi = true;
-                        response = isBucketEmpty(s3Client, container, prefix, true);
-                    } else {
-                        throw ex;
-                    }
-                } catch (SdkException ex2) {
-                    throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-                }
-            } catch (SdkException ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            } finally {
-                if (s3Client != null) {
-                    CleanupUtils.close(s3Client, null);
-                }
-            }
-
-            boolean isEmpty = useOldApi ? ((ListObjectsResponse) response).contents().isEmpty()
-                    : ((ListObjectsV2Response) response).contents().isEmpty();
-            if (isEmpty && collector.shouldWarn()) {
-                Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
-                collector.warn(warning);
-            }
-
-            // Returns 200 only in case the bucket exists, otherwise, throws an exception. However, to
-            // ensure coverage, check if the result is successful as well and not only catch exceptions
-            if (!response.sdkHttpResponse().isSuccessful()) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_CONTAINER_NOT_FOUND, container);
-            }
-        }
-
-        /**
-         * Checks for a single object in the specified bucket to determine if the bucket is empty or not.
-         *
-         * @param s3Client  s3 client
-         * @param container the container name
-         * @param prefix    Prefix to be used
-         * @param useOldApi flag whether to use the old API or not
-         * @return returns the S3 response
-         */
-        private static S3Response isBucketEmpty(S3Client s3Client, String container, String prefix, boolean useOldApi) {
-            S3Response response;
-            if (useOldApi) {
-                ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder();
-                listObjectsBuilder.prefix(prefix);
-                response = s3Client.listObjects(listObjectsBuilder.bucket(container).maxKeys(1).build());
-            } else {
-                ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder();
-                listObjectsBuilder.prefix(prefix);
-                response = s3Client.listObjectsV2(listObjectsBuilder.bucket(container).maxKeys(1).build());
-            }
-            return response;
-        }
-
-        /**
-         * Returns the lists of S3 objects.
-         *
-         * @param configuration         properties
-         * @param includeExcludeMatcher include/exclude matchers to apply
-         */
-        public static List<S3Object> listS3Objects(Map<String, String> configuration,
-                IncludeExcludeMatcher includeExcludeMatcher, IWarningCollector warningCollector)
-                throws CompilationException {
-            // Prepare to retrieve the objects
-            List<S3Object> filesOnly;
-            String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-            S3Client s3Client = buildAwsS3Client(configuration);
-            String prefix = getPrefix(configuration);
-
-            try {
-                filesOnly = listS3Objects(s3Client, container, prefix, includeExcludeMatcher);
-            } catch (S3Exception ex) {
-                // New API is not implemented, try falling back to old API
-                try {
-                    // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
-                    if (ex.awsErrorDetails().errorCode()
-                            .equals(ExternalDataConstants.AwsS3.ERROR_METHOD_NOT_IMPLEMENTED)) {
-                        filesOnly = oldApiListS3Objects(s3Client, container, prefix, includeExcludeMatcher);
-                    } else {
-                        throw ex;
-                    }
-                } catch (SdkException ex2) {
-                    throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-                }
-            } catch (SdkException ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            } finally {
-                if (s3Client != null) {
-                    CleanupUtils.close(s3Client, null);
-                }
-            }
-
-            // Warn if no files are returned
-            if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
-                Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
-                warningCollector.warn(warning);
-            }
-
-            return filesOnly;
-        }
-
-        /**
-         * Uses the latest API to retrieve the objects from the storage.
-         *
-         * @param s3Client              S3 client
-         * @param container             container name
-         * @param prefix                definition prefix
-         * @param includeExcludeMatcher include/exclude matchers to apply
-         */
-        private static List<S3Object> listS3Objects(S3Client s3Client, String container, String prefix,
-                IncludeExcludeMatcher includeExcludeMatcher) {
-            String newMarker = null;
-            List<S3Object> filesOnly = new ArrayList<>();
-
-            ListObjectsV2Response listObjectsResponse;
-            ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder().bucket(container);
-            listObjectsBuilder.prefix(prefix);
-
-            while (true) {
-                // List the objects from the start, or from the last marker in case of truncated result
-                if (newMarker == null) {
-                    listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.build());
-                } else {
-                    listObjectsResponse =
-                            s3Client.listObjectsV2(listObjectsBuilder.continuationToken(newMarker).build());
-                }
-
-                // Collect the paths to files only
-                collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
-                        includeExcludeMatcher.getMatchersList(), filesOnly);
-
-                // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
-                if (!listObjectsResponse.isTruncated()) {
-                    break;
-                } else {
-                    newMarker = listObjectsResponse.nextContinuationToken();
-                }
-            }
-
-            return filesOnly;
-        }
-
-        /**
-         * Uses the old API (in case the new API is not implemented) to retrieve the objects from the storage
-         *
-         * @param s3Client              S3 client
-         * @param container             container name
-         * @param prefix                definition prefix
-         * @param includeExcludeMatcher include/exclude matchers to apply
-         */
-        private static List<S3Object> oldApiListS3Objects(S3Client s3Client, String container, String prefix,
-                IncludeExcludeMatcher includeExcludeMatcher) {
-            String newMarker = null;
-            List<S3Object> filesOnly = new ArrayList<>();
-
-            ListObjectsResponse listObjectsResponse;
-            ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder().bucket(container);
-            listObjectsBuilder.prefix(prefix);
-
-            while (true) {
-                // List the objects from the start, or from the last marker in case of truncated result
-                if (newMarker == null) {
-                    listObjectsResponse = s3Client.listObjects(listObjectsBuilder.build());
-                } else {
-                    listObjectsResponse = s3Client.listObjects(listObjectsBuilder.marker(newMarker).build());
-                }
-
-                // Collect the paths to files only
-                collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
-                        includeExcludeMatcher.getMatchersList(), filesOnly);
-
-                // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
-                if (!listObjectsResponse.isTruncated()) {
-                    break;
-                } else {
-                    newMarker = listObjectsResponse.nextMarker();
-                }
-            }
-
-            return filesOnly;
-        }
-
-        /**
-         * AWS S3 returns all the objects as paths, not differentiating between folder and files. The path is considered
-         * a file if it does not end up with a "/" which is the separator in a folder structure.
-         *
-         * @param s3Objects List of returned objects
-         */
-        private static void collectAndFilterFiles(List<S3Object> s3Objects,
-                BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<S3Object> filesOnly) {
-            for (S3Object object : s3Objects) {
-                // skip folders
-                if (object.key().endsWith("/")) {
-                    continue;
-                }
-
-                // No filter, add file
-                if (predicate.test(matchers, object.key())) {
-                    filesOnly.add(object);
-                }
-            }
-        }
-    }
-
-    /*
-     * Note: Azure Blob and Azure Datalake use identical authentication, so they are using the same properties.
-     * If they end up diverging, then properties for AzureBlob and AzureDataLake need to be created.
-     */
-    public static class Azure {
-        private Azure() {
-            throw new AssertionError("do not instantiate");
-        }
-
-        /**
-         * Builds the Azure storage account using the provided configuration
-         *
-         * @param configuration properties
-         * @return client
-         */
-        public static BlobServiceClient buildAzureBlobClient(IApplicationContext appCtx,
-                Map<String, String> configuration) throws CompilationException {
-            String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
-            String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
-            String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
-            String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
-            String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
-            String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
-            String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
-            String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
-            String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
-            String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
-
-            // Client builder
-            BlobServiceClientBuilder builder = new BlobServiceClientBuilder();
-            int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
-            RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
-            builder.retryOptions(requestRetryOptions);
-
-            // Endpoint is required
-            if (endpoint == null) {
-                throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
-            }
-            builder.endpoint(endpoint);
-
-            // Shared Key
-            if (accountName != null || accountKey != null) {
-                if (accountName == null) {
-                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
-                            ACCOUNT_KEY_FIELD_NAME);
-                }
-
-                if (accountKey == null) {
-                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
-                            ACCOUNT_NAME_FIELD_NAME);
-                }
-
-                Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
-                        MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
-                        CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
-                if (provided.isPresent()) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
-                            ACCOUNT_KEY_FIELD_NAME);
-                }
-                StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
-                builder.credential(credential);
-            }
-
-            // Shared access signature
-            if (sharedAccessSignature != null) {
-                Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
-                        CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
-                        CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
-                if (provided.isPresent()) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
-                            SHARED_ACCESS_SIGNATURE_FIELD_NAME);
-                }
-                AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
-                builder.credential(credential);
-            }
-
-            // Managed Identity auth
-            if (managedIdentityId != null) {
-                Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME,
-                        CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME,
-                        TENANT_ID_FIELD_NAME);
-                if (provided.isPresent()) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
-                            MANAGED_IDENTITY_ID_FIELD_NAME);
-                }
-                builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
-            }
-
-            // Client secret & certificate auth
-            if (clientId != null) {
-                // Both (or neither) client secret and client secret were provided, only one is allowed
-                if ((clientSecret == null) == (clientCertificate == null)) {
-                    if (clientSecret != null) {
-                        throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
-                                CLIENT_CERTIFICATE_FIELD_NAME);
-                    } else {
-                        throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
-                                CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
-                    }
-                }
-
-                // Tenant ID is required
-                if (tenantId == null) {
-                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
-                            CLIENT_ID_FIELD_NAME);
-                }
-
-                // Client certificate password is not allowed if client secret is used
-                if (clientCertificatePassword != null && clientSecret != null) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
-                            CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
-                }
-
-                // Use AD authentication
-                if (clientSecret != null) {
-                    ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
-                    secret.clientId(clientId);
-                    secret.tenantId(tenantId);
-                    secret.clientSecret(clientSecret);
-                    builder.credential(secret.build());
-                } else {
-                    // Certificate
-                    ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
-                    certificate.clientId(clientId);
-                    certificate.tenantId(tenantId);
-                    try {
-                        InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
-                        if (clientCertificatePassword == null) {
-                            Method pemCertificate = ClientCertificateCredentialBuilder.class
-                                    .getDeclaredMethod("pemCertificate", InputStream.class);
-                            pemCertificate.setAccessible(true);
-                            pemCertificate.invoke(certificate, certificateContent);
-                        } else {
-                            Method pemCertificate = ClientCertificateCredentialBuilder.class
-                                    .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
-                            pemCertificate.setAccessible(true);
-                            pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
-                        }
-                    } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
-                        throw new CompilationException(EXTERNAL_SOURCE_ERROR, ex.getMessage());
-                    }
-                    builder.credential(certificate.build());
-                }
-            }
-
-            // If client id is not present, ensure client secret, certificate, tenant id and client certificate
-            // password are not present
-            if (clientId == null) {
-                Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
-                        CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
-                if (provided.isPresent()) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
-                            SHARED_ACCESS_SIGNATURE_FIELD_NAME);
-                }
-            }
-
-            try {
-                return builder.buildClient();
-            } catch (Exception ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            }
-        }
-
-        /**
-         * Builds the Azure data lake storage account using the provided configuration
-         *
-         * @param configuration properties
-         * @return client
-         */
-        public static DataLakeServiceClient buildAzureDatalakeClient(IApplicationContext appCtx,
-                Map<String, String> configuration) throws CompilationException {
-            String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
-            String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
-            String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
-            String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
-            String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
-            String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
-            String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
-            String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
-            String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
-            String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
-
-            // Client builder
-            DataLakeServiceClientBuilder builder = new DataLakeServiceClientBuilder();
-            int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
-            RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
-            builder.retryOptions(requestRetryOptions);
-
-            // Endpoint is required
-            if (endpoint == null) {
-                throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
-            }
-            builder.endpoint(endpoint);
-
-            // Shared Key
-            if (accountName != null || accountKey != null) {
-                if (accountName == null) {
-                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
-                            ACCOUNT_KEY_FIELD_NAME);
-                }
-
-                if (accountKey == null) {
-                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
-                            ACCOUNT_NAME_FIELD_NAME);
-                }
-
-                Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
-                        MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
-                        CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
-                if (provided.isPresent()) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
-                            ACCOUNT_KEY_FIELD_NAME);
-                }
-                StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
-                builder.credential(credential);
-            }
-
-            // Shared access signature
-            if (sharedAccessSignature != null) {
-                Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
-                        CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
-                        CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
-                if (provided.isPresent()) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
-                            SHARED_ACCESS_SIGNATURE_FIELD_NAME);
-                }
-                AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
-                builder.credential(credential);
-            }
-
-            // Managed Identity auth
-            if (managedIdentityId != null) {
-                Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME,
-                        CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME,
-                        TENANT_ID_FIELD_NAME);
-                if (provided.isPresent()) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
-                            MANAGED_IDENTITY_ID_FIELD_NAME);
-                }
-                builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
-            }
-
-            // Client secret & certificate auth
-            if (clientId != null) {
-                // Both (or neither) client secret and client secret were provided, only one is allowed
-                if ((clientSecret == null) == (clientCertificate == null)) {
-                    if (clientSecret != null) {
-                        throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
-                                CLIENT_CERTIFICATE_FIELD_NAME);
-                    } else {
-                        throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
-                                CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
-                    }
-                }
-
-                // Tenant ID is required
-                if (tenantId == null) {
-                    throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
-                            CLIENT_ID_FIELD_NAME);
-                }
-
-                // Client certificate password is not allowed if client secret is used
-                if (clientCertificatePassword != null && clientSecret != null) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
-                            CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
-                }
-
-                // Use AD authentication
-                if (clientSecret != null) {
-                    ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
-                    secret.clientId(clientId);
-                    secret.tenantId(tenantId);
-                    secret.clientSecret(clientSecret);
-                    builder.credential(secret.build());
-                } else {
-                    // Certificate
-                    ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
-                    certificate.clientId(clientId);
-                    certificate.tenantId(tenantId);
-                    try {
-                        InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
-                        if (clientCertificatePassword == null) {
-                            Method pemCertificate = ClientCertificateCredentialBuilder.class
-                                    .getDeclaredMethod("pemCertificate", InputStream.class);
-                            pemCertificate.setAccessible(true);
-                            pemCertificate.invoke(certificate, certificateContent);
-                        } else {
-                            Method pemCertificate = ClientCertificateCredentialBuilder.class
-                                    .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
-                            pemCertificate.setAccessible(true);
-                            pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
-                        }
-                    } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
-                        throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-                    }
-                    builder.credential(certificate.build());
-                }
-            }
-
-            // If client id is not present, ensure client secret, certificate, tenant id and client certificate
-            // password are not present
-            if (clientId == null) {
-                Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
-                        CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
-                if (provided.isPresent()) {
-                    throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
-                            SHARED_ACCESS_SIGNATURE_FIELD_NAME);
-                }
-            }
-
-            try {
-                return builder.buildClient();
-            } catch (Exception ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            }
-        }
-
-        public static List<BlobItem> listBlobItems(BlobServiceClient blobServiceClient,
-                Map<String, String> configuration, IncludeExcludeMatcher includeExcludeMatcher,
-                IWarningCollector warningCollector) throws CompilationException {
-            String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-
-            List<BlobItem> filesOnly = new ArrayList<>();
-
-            // Ensure the validity of include/exclude
-            ExternalDataUtils.validateIncludeExclude(configuration);
-
-            BlobContainerClient blobContainer;
-            try {
-                blobContainer = blobServiceClient.getBlobContainerClient(container);
-
-                // Get all objects in a container and extract the paths to files
-                ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
-                listBlobsOptions.setPrefix(ExternalDataUtils.getPrefix(configuration));
-                Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
-
-                // Collect the paths to files only
-                collectAndFilterBlobFiles(blobItems, includeExcludeMatcher.getPredicate(),
-                        includeExcludeMatcher.getMatchersList(), filesOnly);
-
-                // Warn if no files are returned
-                if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
-                    Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
-                    warningCollector.warn(warning);
-                }
-            } catch (Exception ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            }
-
-            return filesOnly;
-        }
-
-        /**
-         * Collects and filters the files only, and excludes any folders
-         *
-         * @param items     storage items
-         * @param predicate predicate to test with for file filtration
-         * @param matchers  include/exclude matchers to test against
-         * @param filesOnly List containing the files only (excluding folders)
-         */
-        private static void collectAndFilterBlobFiles(Iterable<BlobItem> items,
-                BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<BlobItem> filesOnly) {
-            for (BlobItem item : items) {
-                String uri = item.getName();
-
-                // skip folders
-                if (uri.endsWith("/")) {
-                    continue;
-                }
-
-                // No filter, add file
-                if (predicate.test(matchers, uri)) {
-                    filesOnly.add(item);
-                }
-            }
-        }
-
-        public static List<PathItem> listDatalakePathItems(DataLakeServiceClient client,
-                Map<String, String> configuration, IncludeExcludeMatcher includeExcludeMatcher,
-                IWarningCollector warningCollector) throws CompilationException {
-            String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-
-            List<PathItem> filesOnly = new ArrayList<>();
-
-            // Ensure the validity of include/exclude
-            ExternalDataUtils.validateIncludeExclude(configuration);
-
-            DataLakeFileSystemClient fileSystemClient;
-            try {
-                fileSystemClient = client.getFileSystemClient(container);
-
-                // Get all objects in a container and extract the paths to files
-                ListPathsOptions listOptions = new ListPathsOptions();
-                boolean recursive = Boolean.parseBoolean(configuration.get(RECURSIVE_FIELD_NAME));
-                listOptions.setRecursive(recursive);
-                listOptions.setPath(ExternalDataUtils.getPrefix(configuration, false));
-                PagedIterable<PathItem> pathItems = fileSystemClient.listPaths(listOptions, null);
-
-                // Collect the paths to files only
-                collectAndFilterDatalakeFiles(pathItems, includeExcludeMatcher.getPredicate(),
-                        includeExcludeMatcher.getMatchersList(), filesOnly);
-
-                // Warn if no files are returned
-                if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
-                    Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
-                    warningCollector.warn(warning);
-                }
-            } catch (Exception ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            }
-
-            return filesOnly;
-        }
-
-        /**
-         * Collects and filters the files only, and excludes any folders
-         *
-         * @param items     storage items
-         * @param predicate predicate to test with for file filtration
-         * @param matchers  include/exclude matchers to test against
-         * @param filesOnly List containing the files only (excluding folders)
-         */
-        private static void collectAndFilterDatalakeFiles(Iterable<PathItem> items,
-                BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<PathItem> filesOnly) {
-            for (PathItem item : items) {
-                String uri = item.getName();
-
-                // skip folders
-                if (uri.endsWith("/")) {
-                    continue;
-                }
-
-                // No filter, add file
-                if (predicate.test(matchers, uri)) {
-                    filesOnly.add(item);
-                }
-            }
-        }
-
-        /**
-         * Validate external dataset properties
-         *
-         * @param configuration properties
-         * @throws CompilationException Compilation exception
-         */
-        public static void validateAzureBlobProperties(Map<String, String> configuration, SourceLocation srcLoc,
-                IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
-
-            // check if the format property is present
-            if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
-                throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
-            }
-
-            validateIncludeExclude(configuration);
-
-            // Check if the bucket is present
-            BlobServiceClient blobServiceClient;
-            try {
-                String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-                blobServiceClient = buildAzureBlobClient(appCtx, configuration);
-                BlobContainerClient blobContainer = blobServiceClient.getBlobContainerClient(container);
-
-                // Get all objects in a container and extract the paths to files
-                ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
-                listBlobsOptions.setPrefix(getPrefix(configuration));
-                Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
-
-                if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
-                    Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
-                    collector.warn(warning);
-                }
-            } catch (CompilationException ex) {
-                throw ex;
-            } catch (Exception ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            }
-        }
-
-        /**
-         * Validate external dataset properties
-         *
-         * @param configuration properties
-         * @throws CompilationException Compilation exception
-         */
-        public static void validateAzureDataLakeProperties(Map<String, String> configuration, SourceLocation srcLoc,
-                IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
-
-            // check if the format property is present
-            if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
-                throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
-            }
-
-            validateIncludeExclude(configuration);
-
-            // Check if the bucket is present
-            DataLakeServiceClient dataLakeServiceClient;
-            try {
-                String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-                dataLakeServiceClient = buildAzureDatalakeClient(appCtx, configuration);
-                DataLakeFileSystemClient fileSystemClient = dataLakeServiceClient.getFileSystemClient(container);
-
-                // Get all objects in a container and extract the paths to files
-                ListPathsOptions listPathsOptions = new ListPathsOptions();
-                listPathsOptions.setPath(getPrefix(configuration));
-                Iterable<PathItem> blobItems = fileSystemClient.listPaths(listPathsOptions, null);
-
-                if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
-                    Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
-                    collector.warn(warning);
-                }
-            } catch (CompilationException ex) {
-                throw ex;
-            } catch (Exception ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            }
-        }
-
-        /**
-         * Builds the Azure Blob storage client using the provided configuration
-         *
-         * @param configuration properties
-         * @see <a href="https://docs.microsoft.com/en-us/azure/databricks/data/data-sources/azure/azure-storage">Azure
-         * Blob storage</a>
-         */
-        public static void configureAzureHdfsJobConf(JobConf conf, Map<String, String> configuration, String endPoint) {
-            String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-            String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
-            String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
-
-            //Disable caching S3 FileSystem
-            HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_AZURE_BLOB_PROTOCOL);
-
-            //Key for Hadoop configuration
-            StringBuilder hadoopKey = new StringBuilder();
-            //Value for Hadoop configuration
-            String hadoopValue;
-            if (accountKey != null || sharedAccessSignature != null) {
-                if (accountKey != null) {
-                    hadoopKey.append(HADOOP_AZURE_FS_ACCOUNT_KEY).append('.');
-                    //Set only the AccountKey
-                    hadoopValue = accountKey;
-                } else {
-                    //Use SAS for Hadoop FS as connectionString is provided
-                    hadoopKey.append(HADOOP_AZURE_FS_SAS).append('.');
-                    //Setting the container is required for SAS
-                    hadoopKey.append(container).append('.');
-                    //Set the connection string for SAS
-                    hadoopValue = sharedAccessSignature;
-                }
-                //Set the endPoint, which includes the AccountName
-                hadoopKey.append(endPoint);
-                //Tells Hadoop we are reading from Blob Storage
-                conf.set(hadoopKey.toString(), hadoopValue);
-            }
-        }
-    }
-
-    public static class GCS {
-        private GCS() {
-            throw new AssertionError("do not instantiate");
-
-        }
-
-        //TODO(htowaileb): Add validation step similar to other externals, which also checks if empty bucket
-        //upon creating the external dataset
-
-        /**
-         * Builds the client using the provided configuration
-         *
-         * @param configuration properties
-         * @return clientasterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
-         * @throws CompilationException CompilationException
-         */
-        public static Storage buildClient(Map<String, String> configuration) throws CompilationException {
-            String jsonCredentials = configuration.get(JSON_CREDENTIALS_FIELD_NAME);
-
-            StorageOptions.Builder builder = StorageOptions.newBuilder();
-
-            // Use credentials if available
-            if (jsonCredentials != null) {
-                try (InputStream credentialsStream = new ByteArrayInputStream(jsonCredentials.getBytes())) {
-                    builder.setCredentials(ServiceAccountCredentials.fromStream(credentialsStream));
-                } catch (IOException ex) {
-                    throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-                }
-            }
-
-            return builder.build().getService();
-        }
-
-        /**
-         * Validate external dataset properties
-         *
-         * @param configuration properties
-         * @throws CompilationException Compilation exception
-         */
-        public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
-                IWarningCollector collector) throws CompilationException {
-
-            // check if the format property is present
-            if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
-                throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
-            }
-
-            // parquet is not supported for google cloud storage
-            if (isParquetFormat(configuration)) {
-                throw new CompilationException(INVALID_REQ_PARAM_VAL, srcLoc, KEY_FORMAT,
-                        configuration.get(KEY_FORMAT));
-            }
-
-            validateIncludeExclude(configuration);
-            String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-
-            try {
-                BlobListOption limitOption = BlobListOption.pageSize(1);
-                BlobListOption prefixOption = BlobListOption.prefix(getPrefix(configuration));
-                Storage storage = buildClient(configuration);
-                Page<Blob> items = storage.list(container, limitOption, prefixOption);
-
-                if (!items.iterateAll().iterator().hasNext() && collector.shouldWarn()) {
-                    Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
-                    collector.warn(warning);
-                }
-            } catch (CompilationException ex) {
-                throw ex;
-            } catch (Exception ex) {
-                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
-            }
-        }
-    }
-
     public static int roundUpToNearestFrameSize(int size, int framesize) {
         return ((size / framesize) + 1) * framesize;
     }
@@ -1911,7 +827,7 @@
         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();
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.java
new file mode 100644
index 0000000..79bbbe2
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.aws.s3;
+
+public class S3Constants {
+    private S3Constants() {
+        throw new AssertionError("do not instantiate");
+    }
+
+    public static final String REGION_FIELD_NAME = "region";
+    public static final String INSTANCE_PROFILE_FIELD_NAME = "instanceProfile";
+    public static final String ACCESS_KEY_ID_FIELD_NAME = "accessKeyId";
+    public static final String SECRET_ACCESS_KEY_FIELD_NAME = "secretAccessKey";
+    public static final String SESSION_TOKEN_FIELD_NAME = "sessionToken";
+    public static final String SERVICE_END_POINT_FIELD_NAME = "serviceEndpoint";
+
+    // AWS S3 specific error codes
+    public static final String ERROR_INTERNAL_ERROR = "InternalError";
+    public static final String ERROR_SLOW_DOWN = "SlowDown";
+    public static final String ERROR_METHOD_NOT_IMPLEMENTED = "NotImplemented";
+
+    /*
+     * Hadoop-AWS
+     * AWS connectors for s3 and s3n are deprecated.
+     */
+    public static final String HADOOP_ACCESS_KEY_ID = "fs.s3a.access.key";
+    public static final String HADOOP_SECRET_ACCESS_KEY = "fs.s3a.secret.key";
+    public static final String HADOOP_SESSION_TOKEN = "fs.s3a.session.token";
+    public static final String HADOOP_REGION = "fs.s3a.region";
+    public static final String HADOOP_SERVICE_END_POINT = "fs.s3a.endpoint";
+
+    /*
+     * Internal configurations
+     */
+    //Allows accessing directories as file system path
+    public static final String HADOOP_PATH_STYLE_ACCESS = "fs.s3a.path.style.access";
+    //The number of maximum HTTP connections in connection pool
+    public static final String HADOOP_S3_CONNECTION_POOL_SIZE = "fs.s3a.connection.maximum";
+    //S3 used protocol
+    public static final String HADOOP_S3_PROTOCOL = "s3a";
+
+    //Hadoop credentials provider key
+    public static final String HADOOP_CREDENTIAL_PROVIDER_KEY = "fs.s3a.aws.credentials.provider";
+    //Anonymous credential provider
+    public static final String HADOOP_ANONYMOUS_ACCESS = "org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider";
+    //Temporary credential provider
+    public static final String HADOOP_TEMP_ACCESS = "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Utils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Utils.java
new file mode 100644
index 0000000..6775bf12b
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Utils.java
@@ -0,0 +1,475 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.aws.s3;
+
+import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_PARAM_VALUE_ALLOWED_VALUE;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.S3_REGION_NOT_SUPPORTED;
+import static org.apache.asterix.external.util.ExternalDataUtils.getPrefix;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.*;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiPredicate;
+import java.util.regex.Matcher;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.api.util.CleanupUtils;
+
+import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsSessionCredentials;
+import software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.exception.SdkException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.S3ClientBuilder;
+import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
+import software.amazon.awssdk.services.s3.model.ListObjectsResponse;
+import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
+import software.amazon.awssdk.services.s3.model.ListObjectsV2Response;
+import software.amazon.awssdk.services.s3.model.S3Exception;
+import software.amazon.awssdk.services.s3.model.S3Object;
+import software.amazon.awssdk.services.s3.model.S3Response;
+
+public class S3Utils {
+    private S3Utils() {
+        throw new AssertionError("do not instantiate");
+    }
+
+    public static boolean isRetryableError(String errorCode) {
+        return errorCode.equals(ERROR_INTERNAL_ERROR) || errorCode.equals(ERROR_SLOW_DOWN);
+    }
+
+    /**
+     * Builds the S3 client using the provided configuration
+     *
+     * @param configuration properties
+     * @return S3 client
+     * @throws CompilationException CompilationException
+     */
+    public static S3Client buildAwsS3Client(Map<String, String> configuration) throws CompilationException {
+        // TODO(Hussain): Need to ensure that all required parameters are present in a previous step
+        String instanceProfile = configuration.get(INSTANCE_PROFILE_FIELD_NAME);
+        String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+        String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+        String sessionToken = configuration.get(SESSION_TOKEN_FIELD_NAME);
+        String regionId = configuration.get(REGION_FIELD_NAME);
+        String serviceEndpoint = configuration.get(SERVICE_END_POINT_FIELD_NAME);
+
+        S3ClientBuilder builder = S3Client.builder();
+
+        // Credentials
+        AwsCredentialsProvider credentialsProvider;
+
+        // nothing provided, anonymous authentication
+        if (instanceProfile == null && accessKeyId == null && secretAccessKey == null && sessionToken == null) {
+            credentialsProvider = AnonymousCredentialsProvider.create();
+        } else if (instanceProfile != null) {
+
+            // only "true" value is allowed
+            if (!instanceProfile.equalsIgnoreCase("true")) {
+                throw new CompilationException(INVALID_PARAM_VALUE_ALLOWED_VALUE, INSTANCE_PROFILE_FIELD_NAME, "true");
+            }
+
+            // no other authentication parameters are allowed
+            if (accessKeyId != null) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+                        INSTANCE_PROFILE_FIELD_NAME);
+            }
+            if (secretAccessKey != null) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
+                        INSTANCE_PROFILE_FIELD_NAME);
+            }
+            if (sessionToken != null) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, SESSION_TOKEN_FIELD_NAME,
+                        INSTANCE_PROFILE_FIELD_NAME);
+            }
+            credentialsProvider = InstanceProfileCredentialsProvider.create();
+        } else if (accessKeyId != null || secretAccessKey != null) {
+            // accessKeyId authentication
+            if (accessKeyId == null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+                        SECRET_ACCESS_KEY_FIELD_NAME);
+            }
+            if (secretAccessKey == null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
+                        ACCESS_KEY_ID_FIELD_NAME);
+            }
+
+            // use session token if provided
+            if (sessionToken != null) {
+                credentialsProvider = StaticCredentialsProvider
+                        .create(AwsSessionCredentials.create(accessKeyId, secretAccessKey, sessionToken));
+            } else {
+                credentialsProvider =
+                        StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKeyId, secretAccessKey));
+            }
+        } else {
+            // if only session token is provided, accessKeyId is required
+            throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+                    SESSION_TOKEN_FIELD_NAME);
+        }
+
+        builder.credentialsProvider(credentialsProvider);
+
+        // Validate the region
+        List<Region> regions = S3Client.serviceMetadata().regions();
+        Optional<Region> selectedRegion = regions.stream().filter(region -> region.id().equals(regionId)).findFirst();
+
+        if (selectedRegion.isEmpty()) {
+            throw new CompilationException(S3_REGION_NOT_SUPPORTED, regionId);
+        }
+        builder.region(selectedRegion.get());
+
+        // Validate the service endpoint if present
+        if (serviceEndpoint != null) {
+            try {
+                URI uri = new URI(serviceEndpoint);
+                try {
+                    builder.endpointOverride(uri);
+                } catch (NullPointerException ex) {
+                    throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+                }
+            } catch (URISyntaxException ex) {
+                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR,
+                        String.format("Invalid service endpoint %s", serviceEndpoint));
+            }
+        }
+
+        return builder.build();
+    }
+
+    /**
+     * Builds the S3 client using the provided configuration
+     *
+     * @param configuration      properties
+     * @param numberOfPartitions number of partitions in the cluster
+     */
+    public static void configureAwsS3HdfsJobConf(JobConf conf, Map<String, String> configuration,
+            int numberOfPartitions) {
+        String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+        String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+        String sessionToken = configuration.get(SESSION_TOKEN_FIELD_NAME);
+        String serviceEndpoint = configuration.get(SERVICE_END_POINT_FIELD_NAME);
+
+        //Disable caching S3 FileSystem
+        HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_S3_PROTOCOL);
+
+        /*
+         * Authentication Methods:
+         * 1- Anonymous: no accessKeyId and no secretAccessKey
+         * 2- Temporary: has to provide accessKeyId, secretAccessKey and sessionToken
+         * 3- Private: has to provide accessKeyId and secretAccessKey
+         */
+        if (accessKeyId == null) {
+            //Tells hadoop-aws it is an anonymous access
+            conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_ANONYMOUS_ACCESS);
+        } else {
+            conf.set(HADOOP_ACCESS_KEY_ID, accessKeyId);
+            conf.set(HADOOP_SECRET_ACCESS_KEY, secretAccessKey);
+            if (sessionToken != null) {
+                conf.set(HADOOP_SESSION_TOKEN, sessionToken);
+                //Tells hadoop-aws it is a temporary access
+                conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_TEMP_ACCESS);
+            }
+        }
+
+        /*
+         * This is to allow S3 definition to have path-style form. Should always be true to match the current
+         * way we access files in S3
+         */
+        conf.set(HADOOP_PATH_STYLE_ACCESS, ExternalDataConstants.TRUE);
+
+        /*
+         * Set the size of S3 connection pool to be the number of partitions
+         */
+        conf.set(HADOOP_S3_CONNECTION_POOL_SIZE, String.valueOf(numberOfPartitions));
+
+        if (serviceEndpoint != null) {
+            // Validation of the URL should be done at hadoop-aws level
+            conf.set(HADOOP_SERVICE_END_POINT, serviceEndpoint);
+        } else {
+            //Region is ignored and buckets could be found by the central endpoint
+            conf.set(HADOOP_SERVICE_END_POINT, Constants.CENTRAL_ENDPOINT);
+        }
+    }
+
+    /**
+     * Validate external dataset properties
+     *
+     * @param configuration properties
+     * @throws CompilationException Compilation exception
+     */
+    public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
+            IWarningCollector collector) throws CompilationException {
+
+        // check if the format property is present
+        if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+            throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+        }
+
+        // Both parameters should be passed, or neither should be passed (for anonymous/no auth)
+        String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+        String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+        if (accessKeyId == null || secretAccessKey == null) {
+            // If one is passed, the other is required
+            if (accessKeyId != null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
+                        ACCESS_KEY_ID_FIELD_NAME);
+            } else if (secretAccessKey != null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+                        SECRET_ACCESS_KEY_FIELD_NAME);
+            }
+        }
+
+        validateIncludeExclude(configuration);
+
+        // Check if the bucket is present
+        S3Client s3Client = buildAwsS3Client(configuration);
+        S3Response response;
+        boolean useOldApi = false;
+        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+        String prefix = getPrefix(configuration);
+
+        try {
+            response = isBucketEmpty(s3Client, container, prefix, false);
+        } catch (S3Exception ex) {
+            // Method not implemented, try falling back to old API
+            try {
+                // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
+                if (ex.awsErrorDetails().errorCode().equals(ERROR_METHOD_NOT_IMPLEMENTED)) {
+                    useOldApi = true;
+                    response = isBucketEmpty(s3Client, container, prefix, true);
+                } else {
+                    throw ex;
+                }
+            } catch (SdkException ex2) {
+                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+            }
+        } catch (SdkException ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        } finally {
+            if (s3Client != null) {
+                CleanupUtils.close(s3Client, null);
+            }
+        }
+
+        boolean isEmpty = useOldApi ? ((ListObjectsResponse) response).contents().isEmpty()
+                : ((ListObjectsV2Response) response).contents().isEmpty();
+        if (isEmpty && collector.shouldWarn()) {
+            Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+            collector.warn(warning);
+        }
+
+        // Returns 200 only in case the bucket exists, otherwise, throws an exception. However, to
+        // ensure coverage, check if the result is successful as well and not only catch exceptions
+        if (!response.sdkHttpResponse().isSuccessful()) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_CONTAINER_NOT_FOUND, container);
+        }
+    }
+
+    /**
+     * Checks for a single object in the specified bucket to determine if the bucket is empty or not.
+     *
+     * @param s3Client  s3 client
+     * @param container the container name
+     * @param prefix    Prefix to be used
+     * @param useOldApi flag whether to use the old API or not
+     * @return returns the S3 response
+     */
+    private static S3Response isBucketEmpty(S3Client s3Client, String container, String prefix, boolean useOldApi) {
+        S3Response response;
+        if (useOldApi) {
+            ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder();
+            listObjectsBuilder.prefix(prefix);
+            response = s3Client.listObjects(listObjectsBuilder.bucket(container).maxKeys(1).build());
+        } else {
+            ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder();
+            listObjectsBuilder.prefix(prefix);
+            response = s3Client.listObjectsV2(listObjectsBuilder.bucket(container).maxKeys(1).build());
+        }
+        return response;
+    }
+
+    /**
+     * Returns the lists of S3 objects.
+     *
+     * @param configuration         properties
+     * @param includeExcludeMatcher include/exclude matchers to apply
+     */
+    public static List<S3Object> listS3Objects(Map<String, String> configuration,
+            AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher,
+            IWarningCollector warningCollector) throws CompilationException {
+        // Prepare to retrieve the objects
+        List<S3Object> filesOnly;
+        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+        S3Client s3Client = buildAwsS3Client(configuration);
+        String prefix = getPrefix(configuration);
+
+        try {
+            filesOnly = listS3Objects(s3Client, container, prefix, includeExcludeMatcher);
+        } catch (S3Exception ex) {
+            // New API is not implemented, try falling back to old API
+            try {
+                // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
+                if (ex.awsErrorDetails().errorCode().equals(ERROR_METHOD_NOT_IMPLEMENTED)) {
+                    filesOnly = oldApiListS3Objects(s3Client, container, prefix, includeExcludeMatcher);
+                } else {
+                    throw ex;
+                }
+            } catch (SdkException ex2) {
+                throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+            }
+        } catch (SdkException ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        } finally {
+            if (s3Client != null) {
+                CleanupUtils.close(s3Client, null);
+            }
+        }
+
+        // Warn if no files are returned
+        if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+            Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+            warningCollector.warn(warning);
+        }
+
+        return filesOnly;
+    }
+
+    /**
+     * Uses the latest API to retrieve the objects from the storage.
+     *
+     * @param s3Client              S3 client
+     * @param container             container name
+     * @param prefix                definition prefix
+     * @param includeExcludeMatcher include/exclude matchers to apply
+     */
+    private static List<S3Object> listS3Objects(S3Client s3Client, String container, String prefix,
+            AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher) {
+        String newMarker = null;
+        List<S3Object> filesOnly = new ArrayList<>();
+
+        ListObjectsV2Response listObjectsResponse;
+        ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder().bucket(container);
+        listObjectsBuilder.prefix(prefix);
+
+        while (true) {
+            // List the objects from the start, or from the last marker in case of truncated result
+            if (newMarker == null) {
+                listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.build());
+            } else {
+                listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.continuationToken(newMarker).build());
+            }
+
+            // Collect the paths to files only
+            collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
+                    includeExcludeMatcher.getMatchersList(), filesOnly);
+
+            // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
+            if (!listObjectsResponse.isTruncated()) {
+                break;
+            } else {
+                newMarker = listObjectsResponse.nextContinuationToken();
+            }
+        }
+
+        return filesOnly;
+    }
+
+    /**
+     * Uses the old API (in case the new API is not implemented) to retrieve the objects from the storage
+     *
+     * @param s3Client              S3 client
+     * @param container             container name
+     * @param prefix                definition prefix
+     * @param includeExcludeMatcher include/exclude matchers to apply
+     */
+    private static List<S3Object> oldApiListS3Objects(S3Client s3Client, String container, String prefix,
+            AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher) {
+        String newMarker = null;
+        List<S3Object> filesOnly = new ArrayList<>();
+
+        ListObjectsResponse listObjectsResponse;
+        ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder().bucket(container);
+        listObjectsBuilder.prefix(prefix);
+
+        while (true) {
+            // List the objects from the start, or from the last marker in case of truncated result
+            if (newMarker == null) {
+                listObjectsResponse = s3Client.listObjects(listObjectsBuilder.build());
+            } else {
+                listObjectsResponse = s3Client.listObjects(listObjectsBuilder.marker(newMarker).build());
+            }
+
+            // Collect the paths to files only
+            collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
+                    includeExcludeMatcher.getMatchersList(), filesOnly);
+
+            // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
+            if (!listObjectsResponse.isTruncated()) {
+                break;
+            } else {
+                newMarker = listObjectsResponse.nextMarker();
+            }
+        }
+
+        return filesOnly;
+    }
+
+    /**
+     * AWS S3 returns all the objects as paths, not differentiating between folder and files. The path is considered
+     * a file if it does not end up with a "/" which is the separator in a folder structure.
+     *
+     * @param s3Objects List of returned objects
+     */
+    private static void collectAndFilterFiles(List<S3Object> s3Objects, BiPredicate<List<Matcher>, String> predicate,
+            List<Matcher> matchers, List<S3Object> filesOnly) {
+        for (S3Object object : s3Objects) {
+            // skip folders
+            if (object.key().endsWith("/")) {
+                continue;
+            }
+
+            // No filter, add file
+            if (predicate.test(matchers, object.key())) {
+                filesOnly.add(object);
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureConstants.java
new file mode 100644
index 0000000..9ade27b
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureConstants.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.azure.blob_storage;
+
+/*
+ * Note: Azure Blob and Azure Datalake use identical authentication, so they are using the same properties.
+ * If they end up diverging, then properties for AzureBlob and AzureDataLake need to be created.
+ */
+public class AzureConstants {
+    private AzureConstants() {
+        throw new AssertionError("do not instantiate");
+    }
+
+    /*
+     * Asterix Configuration Keys
+     */
+    public static final String MANAGED_IDENTITY_ID_FIELD_NAME = "managedIdentityId";
+    public static final String ACCOUNT_NAME_FIELD_NAME = "accountName";
+    public static final String ACCOUNT_KEY_FIELD_NAME = "accountKey";
+    public static final String SHARED_ACCESS_SIGNATURE_FIELD_NAME = "sharedAccessSignature";
+    public static final String TENANT_ID_FIELD_NAME = "tenantId";
+    public static final String CLIENT_ID_FIELD_NAME = "clientId";
+    public static final String CLIENT_SECRET_FIELD_NAME = "clientSecret";
+    public static final String CLIENT_CERTIFICATE_FIELD_NAME = "clientCertificate";
+    public static final String CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME = "clientCertificatePassword";
+    public static final String ENDPOINT_FIELD_NAME = "endpoint";
+
+    // Specific Azure data lake property
+    /*
+    The behavior of Data Lake (true file system) is to read the files of the specified prefix only, example:
+    storage/myData/personal/file1.json
+    storage/myData/personal/file2.json
+    storage/myData/file3.json
+    If the prefix used is "myData", then only the file file3.json is read. However, if the property "recursive"
+    is set to "true" when creating the external dataset, then it goes recursively overall the paths, and the result
+    is file1.json, file2.json and file3.json.
+     */
+    public static final String RECURSIVE_FIELD_NAME = "recursive";
+
+    /*
+     * Hadoop-Azure
+     */
+    //Used when accountName and accessKey are provided
+    public static final String HADOOP_AZURE_FS_ACCOUNT_KEY = "fs.azure.account.key";
+    //Used when a connectionString is provided
+    public static final String HADOOP_AZURE_FS_SAS = "fs.azure.sas";
+    public static final String HADOOP_AZURE_BLOB_PROTOCOL = "wasbs";
+    public static final String HADOOP_AZURE_DATALAKE_PROTOCOL = "abfss";
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureUtils.java
new file mode 100644
index 0000000..0dc9ad2
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/azure/blob_storage/AzureUtils.java
@@ -0,0 +1,636 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.azure.blob_storage;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_SOURCE_ERROR;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_NOT_ALLOWED_AT_SAME_TIME;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_REQUIRED;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.external.util.ExternalDataUtils.getFirstNotNull;
+import static org.apache.asterix.external.util.ExternalDataUtils.getPrefix;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.ACCOUNT_KEY_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.ACCOUNT_NAME_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.CLIENT_CERTIFICATE_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.CLIENT_ID_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.CLIENT_SECRET_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.ENDPOINT_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_BLOB_PROTOCOL;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_FS_ACCOUNT_KEY;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.HADOOP_AZURE_FS_SAS;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.MANAGED_IDENTITY_ID_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.RECURSIVE_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.SHARED_ACCESS_SIGNATURE_FIELD_NAME;
+import static org.apache.asterix.external.util.azure.blob_storage.AzureConstants.TENANT_ID_FIELD_NAME;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiPredicate;
+import java.util.regex.Matcher;
+
+import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+
+import com.azure.core.credential.AzureSasCredential;
+import com.azure.core.http.rest.PagedIterable;
+import com.azure.identity.ClientCertificateCredentialBuilder;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.BlobServiceClientBuilder;
+import com.azure.storage.blob.models.BlobItem;
+import com.azure.storage.blob.models.ListBlobsOptions;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import com.azure.storage.common.policy.RequestRetryOptions;
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.DataLakeServiceClientBuilder;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import com.azure.storage.file.datalake.models.PathItem;
+
+public class AzureUtils {
+    private AzureUtils() {
+        throw new AssertionError("do not instantiate");
+    }
+
+    /**
+     * Builds the Azure storage account using the provided configuration
+     *
+     * @param configuration properties
+     * @return client
+     */
+    public static BlobServiceClient buildAzureBlobClient(IApplicationContext appCtx, Map<String, String> configuration)
+            throws CompilationException {
+        String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
+        String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
+        String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
+        String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+        String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
+        String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
+        String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
+        String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
+        String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
+        String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
+
+        // Client builder
+        BlobServiceClientBuilder builder = new BlobServiceClientBuilder();
+        int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
+        RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
+        builder.retryOptions(requestRetryOptions);
+
+        // Endpoint is required
+        if (endpoint == null) {
+            throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
+        }
+        builder.endpoint(endpoint);
+
+        // Shared Key
+        if (accountName != null || accountKey != null) {
+            if (accountName == null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
+                        ACCOUNT_KEY_FIELD_NAME);
+            }
+
+            if (accountKey == null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
+                        ACCOUNT_NAME_FIELD_NAME);
+            }
+
+            Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
+                    MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+                    CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+            if (provided.isPresent()) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+                        ACCOUNT_KEY_FIELD_NAME);
+            }
+            StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
+            builder.credential(credential);
+        }
+
+        // Shared access signature
+        if (sharedAccessSignature != null) {
+            Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
+                    CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
+                    CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+            if (provided.isPresent()) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+                        SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+            }
+            AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
+            builder.credential(credential);
+        }
+
+        // Managed Identity auth
+        if (managedIdentityId != null) {
+            Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+                    CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+            if (provided.isPresent()) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+                        MANAGED_IDENTITY_ID_FIELD_NAME);
+            }
+            builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
+        }
+
+        // Client secret & certificate auth
+        if (clientId != null) {
+            // Both (or neither) client secret and client secret were provided, only one is allowed
+            if ((clientSecret == null) == (clientCertificate == null)) {
+                if (clientSecret != null) {
+                    throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
+                            CLIENT_CERTIFICATE_FIELD_NAME);
+                } else {
+                    throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
+                            CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
+                }
+            }
+
+            // Tenant ID is required
+            if (tenantId == null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
+                        CLIENT_ID_FIELD_NAME);
+            }
+
+            // Client certificate password is not allowed if client secret is used
+            if (clientCertificatePassword != null && clientSecret != null) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
+                        CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
+            }
+
+            // Use AD authentication
+            if (clientSecret != null) {
+                ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
+                secret.clientId(clientId);
+                secret.tenantId(tenantId);
+                secret.clientSecret(clientSecret);
+                builder.credential(secret.build());
+            } else {
+                // Certificate
+                ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
+                certificate.clientId(clientId);
+                certificate.tenantId(tenantId);
+                try {
+                    InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
+                    if (clientCertificatePassword == null) {
+                        Method pemCertificate = ClientCertificateCredentialBuilder.class
+                                .getDeclaredMethod("pemCertificate", InputStream.class);
+                        pemCertificate.setAccessible(true);
+                        pemCertificate.invoke(certificate, certificateContent);
+                    } else {
+                        Method pemCertificate = ClientCertificateCredentialBuilder.class
+                                .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
+                        pemCertificate.setAccessible(true);
+                        pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
+                    }
+                } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
+                    throw new CompilationException(EXTERNAL_SOURCE_ERROR, ex.getMessage());
+                }
+                builder.credential(certificate.build());
+            }
+        }
+
+        // If client id is not present, ensure client secret, certificate, tenant id and client certificate
+        // password are not present
+        if (clientId == null) {
+            Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
+                    CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+            if (provided.isPresent()) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+                        SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+            }
+        }
+
+        try {
+            return builder.buildClient();
+        } catch (Exception ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        }
+    }
+
+    /**
+     * Builds the Azure data lake storage account using the provided configuration
+     *
+     * @param configuration properties
+     * @return client
+     */
+    public static DataLakeServiceClient buildAzureDatalakeClient(IApplicationContext appCtx,
+            Map<String, String> configuration) throws CompilationException {
+        String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
+        String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
+        String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
+        String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+        String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
+        String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
+        String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
+        String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
+        String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
+        String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
+
+        // Client builder
+        DataLakeServiceClientBuilder builder = new DataLakeServiceClientBuilder();
+        int timeout = appCtx.getExternalProperties().getAzureRequestTimeout();
+        RequestRetryOptions requestRetryOptions = new RequestRetryOptions(null, null, timeout, null, null, null);
+        builder.retryOptions(requestRetryOptions);
+
+        // Endpoint is required
+        if (endpoint == null) {
+            throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
+        }
+        builder.endpoint(endpoint);
+
+        // Shared Key
+        if (accountName != null || accountKey != null) {
+            if (accountName == null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
+                        ACCOUNT_KEY_FIELD_NAME);
+            }
+
+            if (accountKey == null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
+                        ACCOUNT_NAME_FIELD_NAME);
+            }
+
+            Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
+                    MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+                    CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+            if (provided.isPresent()) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+                        ACCOUNT_KEY_FIELD_NAME);
+            }
+            StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
+            builder.credential(credential);
+        }
+
+        // Shared access signature
+        if (sharedAccessSignature != null) {
+            Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
+                    CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
+                    CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+            if (provided.isPresent()) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+                        SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+            }
+            AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
+            builder.credential(credential);
+        }
+
+        // Managed Identity auth
+        if (managedIdentityId != null) {
+            Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+                    CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+            if (provided.isPresent()) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+                        MANAGED_IDENTITY_ID_FIELD_NAME);
+            }
+            builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
+        }
+
+        // Client secret & certificate auth
+        if (clientId != null) {
+            // Both (or neither) client secret and client secret were provided, only one is allowed
+            if ((clientSecret == null) == (clientCertificate == null)) {
+                if (clientSecret != null) {
+                    throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
+                            CLIENT_CERTIFICATE_FIELD_NAME);
+                } else {
+                    throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
+                            CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
+                }
+            }
+
+            // Tenant ID is required
+            if (tenantId == null) {
+                throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
+                        CLIENT_ID_FIELD_NAME);
+            }
+
+            // Client certificate password is not allowed if client secret is used
+            if (clientCertificatePassword != null && clientSecret != null) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
+                        CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
+            }
+
+            // Use AD authentication
+            if (clientSecret != null) {
+                ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
+                secret.clientId(clientId);
+                secret.tenantId(tenantId);
+                secret.clientSecret(clientSecret);
+                builder.credential(secret.build());
+            } else {
+                // Certificate
+                ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
+                certificate.clientId(clientId);
+                certificate.tenantId(tenantId);
+                try {
+                    InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
+                    if (clientCertificatePassword == null) {
+                        Method pemCertificate = ClientCertificateCredentialBuilder.class
+                                .getDeclaredMethod("pemCertificate", InputStream.class);
+                        pemCertificate.setAccessible(true);
+                        pemCertificate.invoke(certificate, certificateContent);
+                    } else {
+                        Method pemCertificate = ClientCertificateCredentialBuilder.class
+                                .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
+                        pemCertificate.setAccessible(true);
+                        pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
+                    }
+                } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
+                    throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+                }
+                builder.credential(certificate.build());
+            }
+        }
+
+        // If client id is not present, ensure client secret, certificate, tenant id and client certificate
+        // password are not present
+        if (clientId == null) {
+            Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
+                    CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+            if (provided.isPresent()) {
+                throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+                        SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+            }
+        }
+
+        try {
+            return builder.buildClient();
+        } catch (Exception ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        }
+    }
+
+    public static List<BlobItem> listBlobItems(BlobServiceClient blobServiceClient, Map<String, String> configuration,
+            AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher,
+            IWarningCollector warningCollector) throws CompilationException {
+        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+        List<BlobItem> filesOnly = new ArrayList<>();
+
+        // Ensure the validity of include/exclude
+        validateIncludeExclude(configuration);
+
+        BlobContainerClient blobContainer;
+        try {
+            blobContainer = blobServiceClient.getBlobContainerClient(container);
+
+            // Get all objects in a container and extract the paths to files
+            ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
+            listBlobsOptions.setPrefix(getPrefix(configuration));
+            Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
+
+            // Collect the paths to files only
+            collectAndFilterBlobFiles(blobItems, includeExcludeMatcher.getPredicate(),
+                    includeExcludeMatcher.getMatchersList(), filesOnly);
+
+            // Warn if no files are returned
+            if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+                Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+                warningCollector.warn(warning);
+            }
+        } catch (Exception ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        }
+
+        return filesOnly;
+    }
+
+    /**
+     * Collects and filters the files only, and excludes any folders
+     *
+     * @param items     storage items
+     * @param predicate predicate to test with for file filtration
+     * @param matchers  include/exclude matchers to test against
+     * @param filesOnly List containing the files only (excluding folders)
+     */
+    private static void collectAndFilterBlobFiles(Iterable<BlobItem> items,
+            BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<BlobItem> filesOnly) {
+        for (BlobItem item : items) {
+            String uri = item.getName();
+
+            // skip folders
+            if (uri.endsWith("/")) {
+                continue;
+            }
+
+            // No filter, add file
+            if (predicate.test(matchers, uri)) {
+                filesOnly.add(item);
+            }
+        }
+    }
+
+    public static List<PathItem> listDatalakePathItems(DataLakeServiceClient client, Map<String, String> configuration,
+            AbstractExternalInputStreamFactory.IncludeExcludeMatcher includeExcludeMatcher,
+            IWarningCollector warningCollector) throws CompilationException {
+        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+        List<PathItem> filesOnly = new ArrayList<>();
+
+        // Ensure the validity of include/exclude
+        validateIncludeExclude(configuration);
+
+        DataLakeFileSystemClient fileSystemClient;
+        try {
+            fileSystemClient = client.getFileSystemClient(container);
+
+            // Get all objects in a container and extract the paths to files
+            ListPathsOptions listOptions = new ListPathsOptions();
+            boolean recursive = Boolean.parseBoolean(configuration.get(RECURSIVE_FIELD_NAME));
+            listOptions.setRecursive(recursive);
+            listOptions.setPath(getPrefix(configuration, false));
+            PagedIterable<PathItem> pathItems = fileSystemClient.listPaths(listOptions, null);
+
+            // Collect the paths to files only
+            collectAndFilterDatalakeFiles(pathItems, includeExcludeMatcher.getPredicate(),
+                    includeExcludeMatcher.getMatchersList(), filesOnly);
+
+            // Warn if no files are returned
+            if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+                Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+                warningCollector.warn(warning);
+            }
+        } catch (Exception ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        }
+
+        return filesOnly;
+    }
+
+    /**
+     * Collects and filters the files only, and excludes any folders
+     *
+     * @param items     storage items
+     * @param predicate predicate to test with for file filtration
+     * @param matchers  include/exclude matchers to test against
+     * @param filesOnly List containing the files only (excluding folders)
+     */
+    private static void collectAndFilterDatalakeFiles(Iterable<PathItem> items,
+            BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<PathItem> filesOnly) {
+        for (PathItem item : items) {
+            String uri = item.getName();
+
+            // skip folders
+            if (uri.endsWith("/")) {
+                continue;
+            }
+
+            // No filter, add file
+            if (predicate.test(matchers, uri)) {
+                filesOnly.add(item);
+            }
+        }
+    }
+
+    /**
+     * Validate external dataset properties
+     *
+     * @param configuration properties
+     * @throws CompilationException Compilation exception
+     */
+    public static void validateAzureBlobProperties(Map<String, String> configuration, SourceLocation srcLoc,
+            IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
+
+        // check if the format property is present
+        if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+            throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+        }
+
+        validateIncludeExclude(configuration);
+
+        // Check if the bucket is present
+        BlobServiceClient blobServiceClient;
+        try {
+            String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+            blobServiceClient = buildAzureBlobClient(appCtx, configuration);
+            BlobContainerClient blobContainer = blobServiceClient.getBlobContainerClient(container);
+
+            // Get all objects in a container and extract the paths to files
+            ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
+            listBlobsOptions.setPrefix(getPrefix(configuration));
+            Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
+
+            if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
+                Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+                collector.warn(warning);
+            }
+        } catch (CompilationException ex) {
+            throw ex;
+        } catch (Exception ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        }
+    }
+
+    /**
+     * Validate external dataset properties
+     *
+     * @param configuration properties
+     * @throws CompilationException Compilation exception
+     */
+    public static void validateAzureDataLakeProperties(Map<String, String> configuration, SourceLocation srcLoc,
+            IWarningCollector collector, IApplicationContext appCtx) throws CompilationException {
+
+        // check if the format property is present
+        if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+            throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+        }
+
+        validateIncludeExclude(configuration);
+
+        // Check if the bucket is present
+        DataLakeServiceClient dataLakeServiceClient;
+        try {
+            String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+            dataLakeServiceClient = buildAzureDatalakeClient(appCtx, configuration);
+            DataLakeFileSystemClient fileSystemClient = dataLakeServiceClient.getFileSystemClient(container);
+
+            // Get all objects in a container and extract the paths to files
+            ListPathsOptions listPathsOptions = new ListPathsOptions();
+            listPathsOptions.setPath(getPrefix(configuration));
+            Iterable<PathItem> blobItems = fileSystemClient.listPaths(listPathsOptions, null);
+
+            if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
+                Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+                collector.warn(warning);
+            }
+        } catch (CompilationException ex) {
+            throw ex;
+        } catch (Exception ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        }
+    }
+
+    /**
+     * Builds the Azure Blob storage client using the provided configuration
+     *
+     * @param configuration properties
+     * @see <a href="https://docs.microsoft.com/en-us/azure/databricks/data/data-sources/azure/azure-storage">Azure
+     * Blob storage</a>
+     */
+    public static void configureAzureHdfsJobConf(JobConf conf, Map<String, String> configuration, String endPoint) {
+        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+        String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
+        String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+
+        //Disable caching S3 FileSystem
+        HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_AZURE_BLOB_PROTOCOL);
+
+        //Key for Hadoop configuration
+        StringBuilder hadoopKey = new StringBuilder();
+        //Value for Hadoop configuration
+        String hadoopValue;
+        if (accountKey != null || sharedAccessSignature != null) {
+            if (accountKey != null) {
+                hadoopKey.append(HADOOP_AZURE_FS_ACCOUNT_KEY).append('.');
+                //Set only the AccountKey
+                hadoopValue = accountKey;
+            } else {
+                //Use SAS for Hadoop FS as connectionString is provided
+                hadoopKey.append(HADOOP_AZURE_FS_SAS).append('.');
+                //Setting the container is required for SAS
+                hadoopKey.append(container).append('.');
+                //Set the connection string for SAS
+                hadoopValue = sharedAccessSignature;
+            }
+            //Set the endPoint, which includes the AccountName
+            hadoopKey.append(endPoint);
+            //Tells Hadoop we are reading from Blob Storage
+            conf.set(hadoopKey.toString(), hadoopValue);
+        }
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.java
new file mode 100644
index 0000000..6bf2266
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.google.gcs;
+
+public class GCSConstants {
+    private GCSConstants() {
+        throw new AssertionError("do not instantiate");
+    }
+
+    public static final String JSON_CREDENTIALS_FIELD_NAME = "jsonCredentials";
+    public static final String ENDPOINT_FIELD_NAME = "endpoint";
+
+    /*
+     * Hadoop internal configuration
+     */
+    public static final String HADOOP_GCS_PROTOCOL = "gs";
+
+    // hadoop credentials
+    public static final String HADOOP_AUTH_TYPE = "fs.gs.auth.type";
+    public static final String HADOOP_AUTH_UNAUTHENTICATED = "UNAUTHENTICATED";
+    public static final String HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE = "SERVICE_ACCOUNT_JSON_KEYFILE";
+    public static final String HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE_PATH =
+            "google.cloud.auth.service.account.json.keyfile";
+
+    // gs hadoop parameters
+    public static final String HADOOP_SUPPORT_COMPRESSED = "fs.gs.inputstream.support.gzip.encoding.enable";
+    public static final String HADOOP_ENDPOINT = "fs.gs.storage.root.url";
+    public static final String HADOOP_MAX_REQUESTS_PER_BATCH = "fs.gs.max.requests.per.batch";
+    public static final String HADOOP_BATCH_THREADS = "fs.gs.batch.threads";
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java
new file mode 100644
index 0000000..93dc272
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.google.gcs;
+
+import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_SOURCE_ERROR;
+import static org.apache.asterix.external.util.ExternalDataUtils.getPrefix;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.ENDPOINT_FIELD_NAME;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE_PATH;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_AUTH_TYPE;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_AUTH_UNAUTHENTICATED;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_ENDPOINT;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.HADOOP_GCS_PROTOCOL;
+import static org.apache.asterix.external.util.google.gcs.GCSConstants.JSON_CREDENTIALS_FIELD_NAME;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiPredicate;
+import java.util.regex.Matcher;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+
+import com.google.api.gax.paging.Page;
+import com.google.auth.oauth2.ServiceAccountCredentials;
+import com.google.cloud.BaseServiceException;
+import com.google.cloud.storage.Blob;
+import com.google.cloud.storage.Storage;
+import com.google.cloud.storage.StorageOptions;
+
+public class GCSUtils {
+    private GCSUtils() {
+        throw new AssertionError("do not instantiate");
+
+    }
+
+    /**
+     * Builds the client using the provided configuration
+     *
+     * @param configuration properties
+     * @return clientasterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+     * @throws CompilationException CompilationException
+     */
+    public static Storage buildClient(Map<String, String> configuration) throws CompilationException {
+        String jsonCredentials = configuration.get(JSON_CREDENTIALS_FIELD_NAME);
+        String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
+
+        StorageOptions.Builder builder = StorageOptions.newBuilder();
+
+        // Use credentials if available
+        if (jsonCredentials != null) {
+            try (InputStream credentialsStream = new ByteArrayInputStream(jsonCredentials.getBytes())) {
+                builder.setCredentials(ServiceAccountCredentials.fromStream(credentialsStream));
+            } catch (IOException ex) {
+                throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+            }
+        }
+
+        if (endpoint != null) {
+            builder.setHost(endpoint);
+        }
+
+        return builder.build().getService();
+    }
+
+    /**
+     * Validate external dataset properties
+     *
+     * @param configuration properties
+     * @throws CompilationException Compilation exception
+     */
+    public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
+            IWarningCollector collector) throws CompilationException {
+
+        // check if the format property is present
+        if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+            throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+        }
+
+        validateIncludeExclude(configuration);
+        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+        try {
+            Storage.BlobListOption limitOption = Storage.BlobListOption.pageSize(1);
+            Storage.BlobListOption prefixOption = Storage.BlobListOption.prefix(getPrefix(configuration));
+            Storage storage = buildClient(configuration);
+            Page<Blob> items = storage.list(container, limitOption, prefixOption);
+
+            if (!items.iterateAll().iterator().hasNext() && collector.shouldWarn()) {
+                Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+                collector.warn(warning);
+            }
+        } catch (CompilationException ex) {
+            throw ex;
+        } catch (Exception ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        }
+    }
+
+    public static List<Blob> listItems(Map<String, String> configuration, IncludeExcludeMatcher includeExcludeMatcher,
+            IWarningCollector warningCollector) throws CompilationException {
+        // Prepare to retrieve the objects
+        List<Blob> filesOnly = new ArrayList<>();
+        String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+        Storage gcs = buildClient(configuration);
+        Storage.BlobListOption options = Storage.BlobListOption.prefix(ExternalDataUtils.getPrefix(configuration));
+        Page<Blob> items;
+
+        try {
+            items = gcs.list(container, options);
+        } catch (BaseServiceException ex) {
+            throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+        }
+
+        // Collect the paths to files only
+        collectAndFilterFiles(items, includeExcludeMatcher.getPredicate(), includeExcludeMatcher.getMatchersList(),
+                filesOnly);
+
+        // Warn if no files are returned
+        if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+            Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+            warningCollector.warn(warning);
+        }
+
+        return filesOnly;
+    }
+
+    /**
+     * Excludes paths ending with "/" as that's a directory indicator, we need to return the files only
+     *
+     * @param items List of returned objects
+     */
+    private static void collectAndFilterFiles(Page<Blob> items, BiPredicate<List<Matcher>, String> predicate,
+            List<Matcher> matchers, List<Blob> filesOnly) {
+        for (Blob item : items.iterateAll()) {
+            // skip folders
+            if (item.getName().endsWith("/")) {
+                continue;
+            }
+
+            // No filter, add file
+            if (predicate.test(matchers, item.getName())) {
+                filesOnly.add(item);
+            }
+        }
+    }
+
+    /**
+     * Builds the client using the provided configuration
+     *
+     * @param configuration      properties
+     * @param numberOfPartitions number of partitions in the cluster
+     */
+    public static void configureHdfsJobConf(JobConf conf, Map<String, String> configuration, int numberOfPartitions) {
+        String jsonCredentials = configuration.get(JSON_CREDENTIALS_FIELD_NAME);
+        String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
+
+        // disable caching FileSystem
+        HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_GCS_PROTOCOL);
+
+        // TODO(htowaileb): needs further testing, recommended to disable by gcs-hadoop team
+        conf.set(GCSConstants.HADOOP_SUPPORT_COMPRESSED, ExternalDataConstants.FALSE);
+
+        // TODO(htowaileb): needs further testing
+        // set number of threads
+        //        conf.set(GCSConstants.HADOOP_MAX_REQUESTS_PER_BATCH, String.valueOf(numberOfPartitions));
+        //        conf.set(GCSConstants.HADOOP_BATCH_THREADS, String.valueOf(numberOfPartitions));
+
+        // authentication method
+        // TODO(htowaileb): find a way to pass the content instead of the path to keyfile, this line is temporary
+        Path credentials = Path.of("credentials.json");
+        if (jsonCredentials == null) {
+            // anonymous access
+            conf.set(HADOOP_AUTH_TYPE, HADOOP_AUTH_UNAUTHENTICATED);
+        } else {
+            // TODO(htowaileb) need to pass the file content
+            conf.set(HADOOP_AUTH_TYPE, HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE);
+            conf.set(HADOOP_AUTH_SERVICE_ACCOUNT_JSON_KEY_FILE_PATH, credentials.toAbsolutePath().toString());
+        }
+
+        // set endpoint if provided, default is https://storage.googleapis.com/
+        if (endpoint != null) {
+            conf.set(HADOOP_ENDPOINT, endpoint);
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-external-data/src/main/java/org/checkerframework/checker/nullness/compatqual/NullableDecl.java
similarity index 61%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to asterixdb/asterix-external-data/src/main/java/org/checkerframework/checker/nullness/compatqual/NullableDecl.java
index 6095b26..a4cb401 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/asterixdb/asterix-external-data/src/main/java/org/checkerframework/checker/nullness/compatqual/NullableDecl.java
@@ -16,9 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.checkerframework.checker.nullness.compatqual;
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
 
-select value v from range(1,2) v where v > ?;
+/*
+ * This is a clean room implementation of the NullableDecl interface based
+ * on 3 requirements:
+ * 1. shall be an @interface named NullableDecl
+ * 2. shall be in the org.checkerframework.checker.nullness.compatqual package
+ * 3. the rention policy for the interface shall be RUNTIME
+ */
+@Retention(RetentionPolicy.RUNTIME)
+public @interface NullableDecl {
+}
diff --git a/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
index dceed82..88f3fcb 100644
--- a/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
+++ b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
@@ -26,4 +26,5 @@
 org.apache.asterix.external.input.record.reader.azure.blob.AzureBlobReaderFactory
 org.apache.asterix.external.input.record.reader.azure.datalake.AzureDataLakeReaderFactory
 org.apache.asterix.external.input.record.reader.azure.parquet.AzureBlobParquetReaderFactory
-org.apache.asterix.external.input.record.reader.azure.parquet.AzureDataLakeParquetReaderFactory
\ No newline at end of file
+org.apache.asterix.external.input.record.reader.azure.parquet.AzureDataLakeParquetReaderFactory
+org.apache.asterix.external.input.record.reader.gcs.parquet.GCSParquetReaderFactory
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/AwsS3Test.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/AwsS3Test.java
index 90ea04b..91afbd8 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/AwsS3Test.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/awss3/AwsS3Test.java
@@ -18,8 +18,8 @@
  */
 package org.apache.asterix.external.input.record.reader.awss3;
 
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ERROR_INTERNAL_ERROR;
-import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ERROR_SLOW_DOWN;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.ERROR_INTERNAL_ERROR;
+import static org.apache.asterix.external.util.aws.s3.S3Constants.ERROR_SLOW_DOWN;
 
 import java.lang.reflect.Field;
 import java.lang.reflect.Method;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
index 2594cdd..b1ad073 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
@@ -107,6 +107,7 @@
         CREATE_SYNONYM,
         SYNONYM_DROP,
         VIEW_DROP,
+        ANALYZE,
         COMPACT,
         EXTERNAL_DATASET_REFRESH,
         SUBSCRIBE_FEED,
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
index 95cccb0..19b3cfa 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
@@ -32,7 +32,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.Counter;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 
-public final class LangRewritingContext {
+public class LangRewritingContext {
     private final MetadataProvider metadataProvider;
     private final IWarningCollector warningCollector;
     private final Map<FunctionSignature, FunctionDecl> declaredFunctions;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/AnalyzeStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/AnalyzeStatement.java
new file mode 100644
index 0000000..cbf2c07
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/AnalyzeStatement.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.lang.common.statement;
+
+import java.util.Locale;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.lang.common.util.ExpressionUtils;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.asterix.object.base.AdmBigIntNode;
+import org.apache.asterix.object.base.AdmDoubleNode;
+import org.apache.asterix.object.base.AdmObjectNode;
+import org.apache.asterix.object.base.AdmStringNode;
+import org.apache.asterix.object.base.IAdmNode;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+
+public class AnalyzeStatement extends AbstractStatement {
+
+    private static final String SAMPLE_FIELD_NAME = "sample";
+    private static final String SAMPLE_LOW = "low";
+    private static final String SAMPLE_MEDIUM = "medium";
+    private static final String SAMPLE_HIGH = "high";
+    private static final int SAMPLE_LOW_SIZE = 1063;
+    private static final int SAMPLE_MEDIUM_SIZE = SAMPLE_LOW_SIZE * 4;
+    private static final int SAMPLE_HIGH_SIZE = SAMPLE_MEDIUM_SIZE * 4;
+    private static final int SAMPLE_DEFAULT_SIZE = SAMPLE_LOW_SIZE;
+
+    private static final String SAMPLE_SEED_FIELD_NAME = "sample-seed";
+
+    private final DataverseName dataverseName;
+    private final String datasetName;
+    private final AdmObjectNode options;
+
+    public AnalyzeStatement(DataverseName dataverseName, String datasetName, RecordConstructor options)
+            throws CompilationException {
+        this.dataverseName = dataverseName;
+        this.datasetName = datasetName;
+        this.options = options == null ? null : validateOptions(ExpressionUtils.toNode(options));
+    }
+
+    private static AdmObjectNode validateOptions(AdmObjectNode options) throws CompilationException {
+        for (String fieldName : options.getFieldNames()) {
+            switch (fieldName) {
+                case SAMPLE_FIELD_NAME:
+                case SAMPLE_SEED_FIELD_NAME:
+                    break;
+                default:
+                    throw new CompilationException(ErrorCode.INVALID_PARAM, fieldName);
+            }
+        }
+        return options;
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.ANALYZE;
+    }
+
+    public DataverseName getDataverseName() {
+        return dataverseName;
+    }
+
+    public String getDatasetName() {
+        return datasetName;
+    }
+
+    public int getSampleSize() throws CompilationException {
+        IAdmNode n = getOption(SAMPLE_FIELD_NAME);
+        if (n == null) {
+            return SAMPLE_DEFAULT_SIZE;
+        }
+        switch (n.getType()) {
+            case STRING:
+                String s = ((AdmStringNode) n).get();
+                switch (s.toLowerCase(Locale.ROOT)) {
+                    case SAMPLE_LOW:
+                        return SAMPLE_LOW_SIZE;
+                    case SAMPLE_MEDIUM:
+                        return SAMPLE_MEDIUM_SIZE;
+                    case SAMPLE_HIGH:
+                        return SAMPLE_HIGH_SIZE;
+                    default:
+                        throw new CompilationException(ErrorCode.INVALID_PROPERTY_FORMAT, SAMPLE_FIELD_NAME);
+                }
+            case BIGINT:
+                int v = (int) ((AdmBigIntNode) n).get();
+                if (!isValidSampleSize(v)) {
+                    throw new CompilationException(ErrorCode.INVALID_PROPERTY_FORMAT, SAMPLE_FIELD_NAME);
+                }
+                return v;
+            case DOUBLE:
+                v = (int) ((AdmDoubleNode) n).get();
+                if (!isValidSampleSize(v)) {
+                    throw new CompilationException(ErrorCode.INVALID_PROPERTY_FORMAT, SAMPLE_FIELD_NAME);
+                }
+                return v;
+            default:
+                throw new CompilationException(ErrorCode.WITH_FIELD_MUST_BE_OF_TYPE, SAMPLE_FIELD_NAME,
+                        BuiltinType.ASTRING.getTypeName(), n.getType().toString());
+        }
+    }
+
+    public long getOrCreateSampleSeed() throws AlgebricksException {
+        IAdmNode n = getOption(SAMPLE_SEED_FIELD_NAME);
+        return n != null ? getSampleSeed(n) : createSampleSeed();
+    }
+
+    private long getSampleSeed(IAdmNode n) throws CompilationException {
+        switch (n.getType()) {
+            case BIGINT:
+                return ((AdmBigIntNode) n).get();
+            case DOUBLE:
+                return (long) ((AdmDoubleNode) n).get();
+            case STRING:
+                String s = ((AdmStringNode) n).get();
+                try {
+                    return Long.parseLong(s);
+                } catch (NumberFormatException e) {
+                    throw new CompilationException(ErrorCode.INVALID_PROPERTY_FORMAT, SAMPLE_SEED_FIELD_NAME);
+                }
+            default:
+                throw new CompilationException(ErrorCode.WITH_FIELD_MUST_BE_OF_TYPE, SAMPLE_SEED_FIELD_NAME,
+                        BuiltinType.AINT64.getTypeName(), n.getType().toString());
+        }
+    }
+
+    private long createSampleSeed() {
+        return System.nanoTime() + System.identityHashCode(this);
+    }
+
+    private boolean isValidSampleSize(int v) {
+        return v >= SAMPLE_LOW_SIZE && v <= SAMPLE_HIGH_SIZE;
+    }
+
+    private IAdmNode getOption(String optionName) {
+        return options != null ? options.get(optionName) : null;
+    }
+
+    @Override
+    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+        return visitor.visit(this, arg);
+    }
+
+    @Override
+    public byte getCategory() {
+        return Category.DDL;
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
index 0ddbeb4..c6db0a3 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
@@ -65,6 +65,7 @@
 import org.apache.asterix.lang.common.expression.UnorderedListTypeDefinition;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.statement.AdapterDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeStatement;
 import org.apache.asterix.lang.common.statement.CompactStatement;
 import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
@@ -929,6 +930,14 @@
     }
 
     @Override
+    public Void visit(AnalyzeStatement as, Integer step) throws CompilationException {
+        out.print(skip(step) + "analyze dataset ");
+        out.print(generateFullName(as.getDataverseName(), as.getDatasetName()));
+        out.println(SEMICOLON);
+        return null;
+    }
+
+    @Override
     public Void visit(CompactStatement del, Integer step) throws CompilationException {
         return null;
     }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
index 91fe664..bc8a8d3 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.lang.common.expression.TypeReferenceExpression;
 import org.apache.asterix.lang.common.expression.UnorderedListTypeDefinition;
 import org.apache.asterix.lang.common.statement.AdapterDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeStatement;
 import org.apache.asterix.lang.common.statement.CompactStatement;
 import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
@@ -261,6 +262,11 @@
     }
 
     @Override
+    public R visit(AnalyzeStatement as, T arg) throws CompilationException {
+        return null;
+    }
+
+    @Override
     public R visit(CompactStatement del, T arg) throws CompilationException {
         return null;
     }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
index c2b1311..a5c71ef 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
@@ -42,6 +42,7 @@
 import org.apache.asterix.lang.common.expression.UnorderedListTypeDefinition;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.statement.AdapterDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeStatement;
 import org.apache.asterix.lang.common.statement.CompactStatement;
 import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateAdapterStatement;
@@ -207,6 +208,8 @@
 
     R visit(SynonymDropStatement del, T arg) throws CompilationException;
 
+    R visit(AnalyzeStatement as, T arg) throws CompilationException;
+
     R visit(CompactStatement del, T arg) throws CompilationException;
 
     R visit(ListSliceExpression expression, T arg) throws CompilationException;
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..4352210 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);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index 4e64944..9edd07d 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -76,6 +76,7 @@
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
+import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
 import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.AbstractLangExpression;
 import org.apache.asterix.lang.common.base.AbstractStatement;
@@ -124,6 +125,7 @@
 import org.apache.asterix.lang.common.literal.TrueLiteral;
 import org.apache.asterix.lang.common.parser.ScopeChecker;
 import org.apache.asterix.lang.common.statement.AdapterDropStatement;
+import org.apache.asterix.lang.common.statement.AnalyzeStatement;
 import org.apache.asterix.lang.common.statement.CompactStatement;
 import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
 import org.apache.asterix.lang.common.statement.StartFeedStatement;
@@ -205,7 +207,7 @@
 import org.apache.asterix.lang.sqlpp.util.ExpressionToVariableUtil;
 import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
 import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
-import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
+import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.asterix.om.exceptions.TypeMismatchException;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.BuiltinType;
@@ -836,6 +838,7 @@
     | stmt = UpsertStatement()
     | stmt = ConnectionStatement()
     | stmt = CompactStatement()
+    | stmt = AnalyzeStatement()
     | stmt = Query()
     | stmt = RefreshExternalDatasetStatement()
   )
@@ -1378,7 +1381,7 @@
   <ON> nameComponents = QualifiedName() (<TYPE> <BTREE>)?
   {
     if (indexName == null) {
-      indexName = "primary_idx_" + nameComponents.second;
+      indexName = MetadataConstants.PRIMARY_INDEX_PREFIX + nameComponents.second;
     }
     CreateIndexStatement stmt = new CreateIndexStatement(nameComponents.first, nameComponents.second,
       new Identifier(indexName), IndexType.BTREE, Collections.emptyList(), false, -1, null, ifNotExists, null, null,
@@ -2628,6 +2631,25 @@
     }
 }
 
+Statement AnalyzeStatement() throws ParseException:
+{
+  Token startToken = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
+  RecordConstructor withRecord = null;
+}
+{
+  <ANALYZE> { startToken = token; } Dataset() nameComponents = QualifiedName()
+  ( <WITH> withRecord = RecordConstructor() )?
+  {
+    try {
+    AnalyzeStatement stmt = new AnalyzeStatement(nameComponents.first, nameComponents.second.getValue(), withRecord);
+    return addSourceLocation(stmt, startToken);
+    } catch (CompilationException e) {
+       throw new SqlppParseException(getSourceLocation(startToken), e.getMessage());
+    }
+  }
+}
+
 Statement CompactStatement() throws ParseException:
 {
   Token startToken = null;
@@ -5318,6 +5340,7 @@
 {
   <ADAPTER: "adapter">
   | <ALL : "all">
+  | <ANALYZE: "analyze">
   | <AND : "and">
   | <ANY : "any">
   | <APPLY : "apply">
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index fd0d30b..8880461 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -423,16 +423,16 @@
         if (createMetadataDataset) {
             final double bloomFilterFalsePositiveRate =
                     appContext.getStorageProperties().getBloomFilterFalsePositiveRate();
-            LSMBTreeLocalResourceFactory lsmBtreeFactory =
-                    new LSMBTreeLocalResourceFactory(storageComponentProvider.getStorageManager(), typeTraits,
-                            cmpFactories, null, null, null, opTrackerFactory, ioOpCallbackFactory,
-                            pageWriteCallbackFactory, storageComponentProvider.getMetadataPageManagerFactory(),
-                            new AsterixVirtualBufferCacheProvider(datasetId),
-                            storageComponentProvider.getIoOperationSchedulerProvider(),
-                            appContext.getMetadataMergePolicyFactory(),
-                            StorageConstants.DEFAULT_COMPACTION_POLICY_PROPERTIES, true, bloomFilterKeyFields,
-                            bloomFilterFalsePositiveRate, true, null, NoOpCompressorDecompressorFactory.INSTANCE, true,
-                            TypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
+            LSMBTreeLocalResourceFactory lsmBtreeFactory = new LSMBTreeLocalResourceFactory(
+                    storageComponentProvider.getStorageManager(), typeTraits, cmpFactories, null, null, null,
+                    opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory,
+                    storageComponentProvider.getMetadataPageManagerFactory(),
+                    new AsterixVirtualBufferCacheProvider(datasetId),
+                    storageComponentProvider.getIoOperationSchedulerProvider(),
+                    appContext.getMetadataMergePolicyFactory(), StorageConstants.DEFAULT_COMPACTION_POLICY_PROPERTIES,
+                    true, bloomFilterKeyFields, bloomFilterFalsePositiveRate, true, null,
+                    NoOpCompressorDecompressorFactory.INSTANCE, true,
+                    TypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE, false);
             DatasetLocalResourceFactory dsLocalResourceFactory =
                     new DatasetLocalResourceFactory(datasetId, lsmBtreeFactory);
             // TODO(amoudi) Creating the index should be done through the same code path as
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
index c0bdc75..9a2821e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
@@ -100,7 +100,7 @@
                         pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                         mergePolicyFactory, mergePolicyProperties, true, null, bloomFilterFalsePositiveRate,
                         index.isPrimaryIndex(), btreeFields, compDecompFactory, false,
-                        typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
+                        typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE, false);
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE,
                         dataset.getDatasetType().toString());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
index 168b56e..d119067 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
@@ -21,6 +21,7 @@
 import java.util.List;
 import java.util.Map;
 
+import org.apache.asterix.common.config.DatasetConfig;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import org.apache.asterix.common.context.IStorageComponentProvider;
@@ -92,13 +93,13 @@
                                 filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
                                 ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
                                 ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
-                                bloomFilterFalsePositiveRate, false, btreeFields, hasBloomFilter,
+                                bloomFilterFalsePositiveRate, btreeFields, hasBloomFilter,
                                 typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE)
                         : new ExternalBTreeWithBuddyLocalResourceFactory(storageManager, typeTraits, cmpFactories,
                                 filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
                                 ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
                                 ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
-                                bloomFilterFalsePositiveRate, false, btreeFields, hasBloomFilter,
+                                bloomFilterFalsePositiveRate, btreeFields, hasBloomFilter,
                                 typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
             case INTERNAL:
                 AsterixVirtualBufferCacheProvider vbcProvider =
@@ -112,12 +113,15 @@
                     compDecompFactory = NoOpCompressorDecompressorFactory.INSTANCE;
                 }
 
+                boolean isSecondaryNoIncrementalMaintenance = index.getIndexType() == DatasetConfig.IndexType.SAMPLE;
+
                 return new LSMBTreeLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
                         filterCmpFactories, filterFields, opTrackerFactory, ioOpCallbackFactory,
                         pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                         mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
                         bloomFilterFalsePositiveRate, index.isPrimaryIndex(), btreeFields, compDecompFactory,
-                        hasBloomFilter, typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
+                        hasBloomFilter, typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE,
+                        isSecondaryNoIncrementalMaintenance);
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE,
                         dataset.getDatasetType().toString());
@@ -127,7 +131,7 @@
     private static ITypeTraits[] getTypeTraits(MetadataProvider metadataProvider, Dataset dataset, Index index,
             ARecordType recordType, ARecordType metaType) throws AlgebricksException {
         ITypeTraits[] primaryTypeTraits = dataset.getPrimaryTypeTraits(metadataProvider, recordType, metaType);
-        if (index.isPrimaryIndex()) {
+        if (index.isPrimaryIndex() || index.getIndexType() == DatasetConfig.IndexType.SAMPLE) {
             return primaryTypeTraits;
         } else if (dataset.getDatasetType() == DatasetType.EXTERNAL
                 && index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
@@ -162,7 +166,7 @@
             Index index, ARecordType recordType, ARecordType metaType) throws AlgebricksException {
         IBinaryComparatorFactory[] primaryCmpFactories =
                 dataset.getPrimaryComparatorFactories(metadataProvider, recordType, metaType);
-        if (index.isPrimaryIndex()) {
+        if (index.isPrimaryIndex() || index.getIndexType() == DatasetConfig.IndexType.SAMPLE) {
             return dataset.getPrimaryComparatorFactories(metadataProvider, recordType, metaType);
         } else if (dataset.getDatasetType() == DatasetType.EXTERNAL
                 && index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
@@ -200,7 +204,8 @@
         if (index.isPrimaryIndex() || index.isPrimaryKeyIndex()) {
             return dataset.getPrimaryBloomFilterFields();
         }
-        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL
+                && index.getIndexType() != DatasetConfig.IndexType.SAMPLE) {
             if (index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
                 return FilesIndexDescription.BLOOM_FILTER_FIELDS;
             } else {
@@ -224,6 +229,8 @@
                     bloomFilterKeyFields[i] = i;
                 }
                 return bloomFilterKeyFields;
+            case SAMPLE:
+                return null;
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
                         String.valueOf(index.getIndexType()));
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
index 454d3da..95b0906 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
@@ -60,6 +60,7 @@
         public static final byte FEED = 0x02;
         public static final byte LOADABLE = 0x03;
         public static final byte FUNCTION = 0x04;
+        public static final byte SAMPLE = 0x05;
 
         // Hide implicit public constructor
         private Type() {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java
index 21b744f..324c660 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSourcePartitioningProvider.java
@@ -61,6 +61,7 @@
                 pp = getFeedPartitioningProperty(ds, domain, scanVariables);
                 break;
             case DataSource.Type.INTERNAL_DATASET:
+            case DataSource.Type.SAMPLE:
                 Set<LogicalVariable> pvars = new ListSet<>();
                 pp = getInternalDatasetPartitioningProperty(ds, domain, scanVariables, pvars);
                 propsLocal.add(new LocalOrderProperty(getOrderColumns(pvars)));
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
index 1d82b19..5daaaa0 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
@@ -53,6 +53,7 @@
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
 
 public class DatasetDataSource extends DataSource {
 
@@ -78,13 +79,18 @@
 
     private void initInternalDataset(IAType itemType, IAType metaItemType, IDatasetDetails datasetDetails)
             throws AlgebricksException {
-        InternalDatasetDetails internalDatasetDetails = (InternalDatasetDetails) datasetDetails;
+        schemaTypes =
+                createSchemaTypesForInternalDataset(itemType, metaItemType, (InternalDatasetDetails) datasetDetails);
+    }
+
+    static IAType[] createSchemaTypesForInternalDataset(IAType itemType, IAType metaItemType,
+            InternalDatasetDetails internalDatasetDetails) throws AlgebricksException {
         ARecordType recordType = (ARecordType) itemType;
         ARecordType metaRecordType = (ARecordType) metaItemType;
         List<IAType> partitioningKeyTypes =
                 KeyFieldTypeUtil.getPartitioningKeyTypes(internalDatasetDetails, recordType, metaRecordType);
         int n = partitioningKeyTypes.size();
-        schemaTypes = metaItemType == null ? new IAType[n + 1] : new IAType[n + 2];
+        IAType[] schemaTypes = metaItemType == null ? new IAType[n + 1] : new IAType[n + 2];
         for (int keyIndex = 0; keyIndex < n; ++keyIndex) {
             schemaTypes[keyIndex] = partitioningKeyTypes.get(keyIndex);
         }
@@ -92,11 +98,17 @@
         if (metaItemType != null) {
             schemaTypes[n + 1] = metaItemType;
         }
+        return schemaTypes;
     }
 
     private void initExternalDataset(IAType itemType) {
-        schemaTypes = new IAType[1];
+        schemaTypes = createSchemaTypesForExternalDataset(itemType);
+    }
+
+    static IAType[] createSchemaTypesForExternalDataset(IAType itemType) {
+        IAType[] schemaTypes = new IAType[1];
         schemaTypes[0] = itemType;
+        return schemaTypes;
     }
 
     @Override
@@ -135,7 +147,7 @@
                 return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false, null,
                         ((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null, true,
                         true, false, null, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory,
-                        outputLimit, false, false);
+                        outputLimit, false, false, DefaultTupleProjectorFactory.INSTANCE);
             default:
                 throw new AlgebricksException("Unknown datasource type");
         }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index 4cc83ab..87959c4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -93,7 +93,9 @@
 import org.apache.asterix.metadata.lock.ExternalDatasetsRegistry;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.FullTextUtil;
+import org.apache.asterix.metadata.utils.IndexUtil;
 import org.apache.asterix.metadata.utils.MetadataConstants;
+import org.apache.asterix.metadata.utils.MetadataUtil;
 import org.apache.asterix.metadata.utils.SplitsAndConstraintsUtil;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionExtensionManager;
@@ -115,6 +117,7 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.core.algebra.base.Counter;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
@@ -169,6 +172,7 @@
 import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import org.apache.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
 import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
 
 public class MetadataProvider implements IMetadataProvider<DataSourceId, String> {
 
@@ -182,12 +186,11 @@
     private Dataverse defaultDataverse;
     private MetadataTransactionContext mdTxnCtx;
     private boolean isWriteTransaction;
-    private IAWriterFactory writerFactory;
     private FileSplit outputFile;
     private boolean asyncResults;
     private long maxResultReads;
     private ResultSetId resultSetId;
-    private IResultSerializerFactoryProvider resultSerializerFactoryProvider;
+    private Counter resultSetIdCounter;
     private TxnId txnId;
     private Map<String, Integer> externalDataLocks;
     private boolean blockingOperatorDisabled = false;
@@ -262,10 +265,6 @@
         this.isWriteTransaction = writeTransaction;
     }
 
-    public void setWriterFactory(IAWriterFactory writerFactory) {
-        this.writerFactory = writerFactory;
-    }
-
     public void setMetadataTxnContext(MetadataTransactionContext mdTxnCtx) {
         this.mdTxnCtx = mdTxnCtx;
     }
@@ -274,10 +273,6 @@
         return mdTxnCtx;
     }
 
-    public IAWriterFactory getWriterFactory() {
-        return this.writerFactory;
-    }
-
     public FileSplit getOutputFile() {
         return outputFile;
     }
@@ -310,12 +305,12 @@
         this.resultSetId = resultSetId;
     }
 
-    public void setResultSerializerFactoryProvider(IResultSerializerFactoryProvider rafp) {
-        this.resultSerializerFactoryProvider = rafp;
+    public Counter getResultSetIdCounter() {
+        return resultSetIdCounter;
     }
 
-    public IResultSerializerFactoryProvider getResultSerializerFactoryProvider() {
-        return resultSerializerFactoryProvider;
+    public void setResultSetIdCounter(Counter resultSetIdCounter) {
+        this.resultSetIdCounter = resultSetIdCounter;
     }
 
     public boolean isWriteTransaction() {
@@ -437,6 +432,16 @@
         return MetadataManagerUtil.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
     }
 
+    public Index findSampleIndex(DataverseName dataverseName, String datasetName) throws AlgebricksException {
+        Pair<String, String> sampleIndexNames = IndexUtil.getSampleIndexNames(datasetName);
+        Index sampleIndex = getIndex(dataverseName, datasetName, sampleIndexNames.first);
+        if (sampleIndex != null && sampleIndex.getPendingOp() == MetadataUtil.PENDING_NO_OP) {
+            return sampleIndex;
+        }
+        sampleIndex = getIndex(dataverseName, datasetName, sampleIndexNames.second);
+        return sampleIndex != null && sampleIndex.getPendingOp() == MetadataUtil.PENDING_NO_OP ? sampleIndex : null;
+    }
+
     public Triple<DataverseName, String, Boolean> resolveDatasetNameUsingSynonyms(DataverseName dataverseName,
             String datasetName, boolean includingViews) throws AlgebricksException {
         DataverseName dvName = getActiveDataverseName(dataverseName);
@@ -542,7 +547,8 @@
             int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
             boolean propagateFilter, IMissingWriterFactory nonFilterWriterFactory, int[] minFilterFieldIndexes,
             int[] maxFilterFieldIndexes, ITupleFilterFactory tupleFilterFactory, long outputLimit,
-            boolean isIndexOnlyPlan, boolean isPrimaryIndexPointSearch) throws AlgebricksException {
+            boolean isIndexOnlyPlan, boolean isPrimaryIndexPointSearch, ITupleProjectorFactory tupleProjectorFactory)
+            throws AlgebricksException {
         boolean isSecondary = true;
         Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
                 dataset.getDatasetName(), dataset.getDatasetName());
@@ -561,6 +567,12 @@
             case BTREE:
                 numSecondaryKeys = ((Index.ValueIndexDetails) theIndex.getIndexDetails()).getKeyFieldNames().size();
                 break;
+            case SAMPLE:
+                if (isIndexOnlyPlan) {
+                    throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, "");
+                }
+                numSecondaryKeys = 0;
+                break;
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
                         theIndex.getIndexType().toString());
@@ -601,12 +613,13 @@
                     ? new LSMBTreeBatchPointSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
                             highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput,
                             retainMissing, nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
-                            maxFilterFieldIndexes, tupleFilterFactory, outputLimit)
+                            maxFilterFieldIndexes, tupleFilterFactory, outputLimit, tupleProjectorFactory)
                     : new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields, highKeyFields,
                             lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
                             nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes,
                             propagateFilter, nonFilterWriterFactory, tupleFilterFactory, outputLimit,
-                            proceedIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
+                            proceedIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan,
+                            tupleProjectorFactory);
         } else {
             btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
                     highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
@@ -683,7 +696,8 @@
 
     @Override
     public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteFileRuntime(IDataSink sink,
-            int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc) {
+            int[] printColumns, IPrinterFactory[] printerFactories, IAWriterFactory writerFactory,
+            RecordDescriptor inputDesc) {
         FileSplitDataSink fsds = (FileSplitDataSink) sink;
         FileSplitSinkId fssi = fsds.getId();
         FileSplit fs = fssi.getFileSplit();
@@ -691,14 +705,15 @@
         String nodeId = fs.getNodeName();
 
         SinkWriterRuntimeFactory runtime =
-                new SinkWriterRuntimeFactory(printColumns, printerFactories, outFile, getWriterFactory(), inputDesc);
+                new SinkWriterRuntimeFactory(printColumns, printerFactories, outFile, writerFactory, inputDesc);
         AlgebricksPartitionConstraint apc = new AlgebricksAbsolutePartitionConstraint(new String[] { nodeId });
         return new Pair<>(runtime, apc);
     }
 
     @Override
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink,
-            int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc,
+            int[] printColumns, IPrinterFactory[] printerFactories, IAWriterFactory writerFactory,
+            IResultSerializerFactoryProvider resultSerializerFactoryProvider, RecordDescriptor inputDesc,
             IResultMetadata metadata, JobSpecification spec) throws AlgebricksException {
         ResultSetDataSink rsds = (ResultSetDataSink) sink;
         ResultSetSinkId rssId = rsds.getId();
@@ -706,7 +721,7 @@
         ResultWriterOperatorDescriptor resultWriter = null;
         try {
             IResultSerializerFactory resultSerializedAppenderFactory = resultSerializerFactoryProvider
-                    .getAqlResultSerializerFactoryProvider(printColumns, printerFactories, getWriterFactory());
+                    .getResultSerializerFactoryProvider(printColumns, printerFactories, writerFactory);
             resultWriter = new ResultWriterOperatorDescriptor(spec, rsId, metadata, getResultAsyncMode(),
                     resultSerializedAppenderFactory, getMaxResultReads());
         } catch (IOException e) {
@@ -859,7 +874,7 @@
      *
      * @param dataset
      * @return Number of elements that will be used to create a bloom filter per
-     *         dataset per partition
+     * dataset per partition
      * @throws AlgebricksException
      */
     public long getCardinalityPerPartitionHint(Dataset dataset) throws AlgebricksException {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/SampleDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/SampleDataSource.java
new file mode 100644
index 0000000..418cb8e
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/SampleDataSource.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.metadata.declared;
+
+import java.util.List;
+
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.InternalDatasetDetails;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
+
+public class SampleDataSource extends DataSource {
+
+    private final Dataset dataset;
+
+    private final String sampleIndexName;
+
+    public SampleDataSource(Dataset dataset, String sampleIndexName, IAType itemType, IAType metaItemType,
+            INodeDomain domain) throws AlgebricksException {
+        super(createSampleDataSourceId(dataset, sampleIndexName), itemType, metaItemType, Type.SAMPLE, domain);
+        this.dataset = dataset;
+        this.sampleIndexName = sampleIndexName;
+        this.schemaTypes = DatasetDataSource.createSchemaTypesForInternalDataset(itemType, metaItemType,
+                (InternalDatasetDetails) dataset.getDatasetDetails());
+    }
+
+    @Override
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
+            MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
+            List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
+            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+            ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
+            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
+            IProjectionInfo<?> projectionInfo) throws AlgebricksException {
+        return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false, null, dataset,
+                sampleIndexName, null, null, true, true, false, null, null, null, tupleFilterFactory, outputLimit,
+                false, false, DefaultTupleProjectorFactory.INSTANCE);
+    }
+
+    @Override
+    public boolean isScanAccessPathALeaf() {
+        return false;
+    }
+
+    private static DataSourceId createSampleDataSourceId(Dataset dataset, String sampleIndexName) {
+        return new DataSourceId(dataset.getDataverseName(), dataset.getDatasetName(), new String[] { sampleIndexName });
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index 37e22ed..e3935a4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -497,6 +497,10 @@
                         recordType, metaType, mergePolicyFactory, mergePolicyProperties, filterTypeTraits,
                         filterCmpFactories);
                 break;
+            case SAMPLE:
+                resourceFactory = BTreeResourceFactoryProvider.INSTANCE.getResourceFactory(mdProvider, this, index,
+                        recordType, metaType, mergePolicyFactory, mergePolicyProperties, null, null);
+                break;
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
                         index.getIndexType().toString());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
index 9e67292..21d2aaa 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
@@ -275,6 +275,7 @@
         switch (indexType) {
             case ARRAY:
             case BTREE:
+            case SAMPLE:
                 return ResourceType.LSM_BTREE;
             case RTREE:
                 return ResourceType.LSM_RTREE;
@@ -296,7 +297,8 @@
     public enum IndexCategory {
         VALUE,
         TEXT,
-        ARRAY;
+        ARRAY,
+        SAMPLE;
 
         public static IndexCategory of(IndexType indexType) {
             switch (indexType) {
@@ -310,6 +312,8 @@
                     return TEXT;
                 case ARRAY:
                     return ARRAY;
+                case SAMPLE:
+                    return SAMPLE;
                 default:
                     throw new IllegalArgumentException(String.valueOf(indexType));
             }
@@ -533,6 +537,75 @@
         }
     }
 
+    public static class SampleIndexDetails extends AbstractIndexDetails {
+
+        private static final long serialVersionUID = 1L;
+
+        private final List<List<String>> keyFieldNames;
+
+        private final List<Integer> keyFieldSourceIndicators;
+
+        private final List<IAType> keyFieldTypes;
+
+        private final int sampleCardinalityTarget;
+
+        private final long sourceCardinality;
+
+        private final int sourceAvgItemSize;
+
+        private final long sampleSeed;
+
+        public SampleIndexDetails(List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators,
+                List<IAType> keyFieldTypes, int sampleCardinalityTarget, long sourceCardinality, int sourceAvgItemSize,
+                long sampleSeed) {
+            this.keyFieldNames = keyFieldNames;
+            this.keyFieldSourceIndicators = keyFieldSourceIndicators;
+            this.keyFieldTypes = keyFieldTypes;
+            this.sampleCardinalityTarget = sampleCardinalityTarget;
+            this.sourceCardinality = sourceCardinality;
+            this.sourceAvgItemSize = sourceAvgItemSize;
+            this.sampleSeed = sampleSeed;
+        }
+
+        @Override
+        IndexCategory getIndexCategory() {
+            return IndexCategory.SAMPLE;
+        }
+
+        public List<List<String>> getKeyFieldNames() {
+            return keyFieldNames;
+        }
+
+        public List<Integer> getKeyFieldSourceIndicators() {
+            return keyFieldSourceIndicators;
+        }
+
+        public List<IAType> getKeyFieldTypes() {
+            return keyFieldTypes;
+        }
+
+        @Override
+        public boolean isOverridingKeyFieldTypes() {
+            return false;
+        }
+
+        public int getSampleCardinalityTarget() {
+            return sampleCardinalityTarget;
+        }
+
+        public long getSourceCardinality() {
+            return sourceCardinality;
+        }
+
+        public int getSourceAvgItemSize() {
+            return sourceAvgItemSize;
+        }
+
+        public long getSampleSeed() {
+            return sampleSeed;
+        }
+    }
+
     @Deprecated
     private static Index.IIndexDetails createSimpleIndexDetails(IndexType indexType, List<List<String>> keyFieldNames,
             List<Integer> keyFieldSourceIndicators, List<IAType> keyFieldTypes, boolean overrideKeyFieldTypes,
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index bb51748..9c742ed 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -47,6 +47,7 @@
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.ACollectionCursor;
 import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AInt64;
 import org.apache.asterix.om.base.AInt8;
 import org.apache.asterix.om.base.AMutableInt8;
 import org.apache.asterix.om.base.ANull;
@@ -90,6 +91,10 @@
     public static final String INDEX_SEARCHKEY_ELEMENTS_FIELD_NAME = "SearchKeyElements";
     public static final String COMPLEXSEARCHKEY_UNNEST_FIELD_NAME = "UnnestList";
     public static final String COMPLEXSEARCHKEY_PROJECT_FIELD_NAME = "ProjectList";
+    public static final String SAMPLE_SEED = "SampleSeed";
+    public static final String SAMPLE_CARDINALITY_TARGET = "SampleCardinalityTarget";
+    public static final String SOURCE_CARDINALITY = "SourceCardinality";
+    public static final String SOURCE_AVG_ITEM_SIZE = "SourceAvgItemSize";
 
     protected final TxnId txnId;
     protected final MetadataNode metadataNode;
@@ -240,6 +245,9 @@
                     searchElements.add(searchElement);
                 }
                 break;
+            case SAMPLE:
+                searchElements = Collections.emptyList();
+                break;
             default:
                 throw new AsterixException(ErrorCode.METADATA_ERROR, indexType.toString());
         }
@@ -452,6 +460,39 @@
                 }
                 indexDetails = new Index.ArrayIndexDetails(elementList, isOverridingKeyTypes);
                 break;
+            case SAMPLE:
+                keyFieldNames =
+                        searchElements.stream().map(Pair::getSecond).map(l -> l.get(0)).collect(Collectors.toList());
+                keyFieldTypes = searchKeyType.stream().map(l -> l.get(0)).collect(Collectors.toList());
+
+                int sampleSeedPos = indexRecord.getType().getFieldIndex(SAMPLE_SEED);
+                if (sampleSeedPos < 0) {
+                    throw new AsterixException(ErrorCode.METADATA_ERROR, SAMPLE_SEED);
+                }
+                long sampleSeed = ((AInt64) indexRecord.getValueByPos(sampleSeedPos)).getLongValue();
+
+                int sampleCardinalityTargetPos = indexRecord.getType().getFieldIndex(SAMPLE_CARDINALITY_TARGET);
+                if (sampleCardinalityTargetPos < 0) {
+                    throw new AsterixException(ErrorCode.METADATA_ERROR, SAMPLE_CARDINALITY_TARGET);
+                }
+                int sampleCardinalityTarget =
+                        ((AInt32) indexRecord.getValueByPos(sampleCardinalityTargetPos)).getIntegerValue();
+
+                int sourceCardinalityPos = indexRecord.getType().getFieldIndex(SOURCE_CARDINALITY);
+                if (sourceCardinalityPos < 0) {
+                    throw new AsterixException(ErrorCode.METADATA_ERROR, SOURCE_CARDINALITY);
+                }
+                long sourceCardinality = ((AInt64) indexRecord.getValueByPos(sourceCardinalityPos)).getLongValue();
+
+                int sourceAvgItemSizePos = indexRecord.getType().getFieldIndex(SOURCE_AVG_ITEM_SIZE);
+                if (sourceAvgItemSizePos < 0) {
+                    throw new AsterixException(ErrorCode.METADATA_ERROR, SOURCE_AVG_ITEM_SIZE);
+                }
+                int sourceAvgItemSize = ((AInt32) indexRecord.getValueByPos(sourceAvgItemSizePos)).getIntegerValue();
+
+                indexDetails = new Index.SampleIndexDetails(keyFieldNames, keyFieldSourceIndicator, keyFieldTypes,
+                        sampleCardinalityTarget, sourceCardinality, sourceAvgItemSize, sampleSeed);
+                break;
             default:
                 throw new AsterixException(ErrorCode.METADATA_ERROR, indexType.toString());
         }
@@ -527,6 +568,9 @@
                 // If we have a complex index, we persist all of the names in the complex SK name array instead.
                 searchKey = Collections.emptyList();
                 break;
+            case SAMPLE:
+                searchKey = ((Index.SampleIndexDetails) index.getIndexDetails()).getKeyFieldNames();
+                break;
             default:
                 throw new AsterixException(ErrorCode.METADATA_ERROR, indexType.toString());
         }
@@ -596,6 +640,7 @@
         writeSearchKeySourceIndicator(index);
         writeExcludeUnknownKey(index);
         writeCast(index);
+        writeSampleDetails(index);
     }
 
     private void writeComplexSearchKeys(Index.ArrayIndexDetails indexDetails) throws HyracksDataException {
@@ -767,6 +812,9 @@
                 keySourceIndicator = ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
                         .map(Index.ArrayIndexElement::getSourceIndicator).collect(Collectors.toList());
                 break;
+            case SAMPLE:
+                keySourceIndicator = ((Index.SampleIndexDetails) index.getIndexDetails()).getKeyFieldSourceIndicators();
+                break;
             default:
                 throw new AsterixException(ErrorCode.METADATA_ERROR, index.getIndexType().toString());
         }
@@ -855,4 +903,38 @@
             }
         }
     }
+
+    private void writeSampleDetails(Index index) throws HyracksDataException {
+        if (index.getIndexType() == IndexType.SAMPLE) {
+            Index.SampleIndexDetails indexDetails = (Index.SampleIndexDetails) index.getIndexDetails();
+
+            nameValue.reset();
+            fieldValue.reset();
+            aString.setValue(SAMPLE_SEED);
+            stringSerde.serialize(aString, nameValue.getDataOutput());
+            int64Serde.serialize(new AInt64(indexDetails.getSampleSeed()), fieldValue.getDataOutput());
+            recordBuilder.addField(nameValue, fieldValue);
+
+            nameValue.reset();
+            fieldValue.reset();
+            aString.setValue(SAMPLE_CARDINALITY_TARGET);
+            stringSerde.serialize(aString, nameValue.getDataOutput());
+            int32Serde.serialize(new AInt32(indexDetails.getSampleCardinalityTarget()), fieldValue.getDataOutput());
+            recordBuilder.addField(nameValue, fieldValue);
+
+            nameValue.reset();
+            fieldValue.reset();
+            aString.setValue(SOURCE_CARDINALITY);
+            stringSerde.serialize(aString, nameValue.getDataOutput());
+            int64Serde.serialize(new AInt64(indexDetails.getSourceCardinality()), fieldValue.getDataOutput());
+            recordBuilder.addField(nameValue, fieldValue);
+
+            nameValue.reset();
+            fieldValue.reset();
+            aString.setValue(SOURCE_AVG_ITEM_SIZE);
+            stringSerde.serialize(aString, nameValue.getDataOutput());
+            int32Serde.serialize(new AInt32(indexDetails.getSourceAvgItemSize()), fieldValue.getDataOutput());
+            recordBuilder.addField(nameValue, fieldValue);
+        }
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
index 92390a7..0ee9516 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
@@ -30,7 +30,6 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.MetadataException;
-import org.apache.asterix.common.exceptions.NoOpWarningCollector;
 import org.apache.asterix.common.external.IDataSourceAdapter;
 import org.apache.asterix.common.external.IDataSourceAdapter.AdapterType;
 import org.apache.asterix.common.functions.ExternalFunctionLanguage;
@@ -62,6 +61,7 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
 
 /**
  * A utility class for providing helper functions for feeds TODO: Refactor this
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
index 5fd96a6..ce98008 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
@@ -174,7 +174,7 @@
         if (fnKind != FunctionKind.SCALAR) {
             return false;
         }
-        if (!(fnInfo instanceof IExternalFunctionInfo)) {
+        if (!fnInfo.isExternal()) {
             return false;
         }
         ExternalFunctionLanguage language = ((IExternalFunctionInfo) fnInfo).getLanguage();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
index b899e16..c6eafe0 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
@@ -30,7 +30,6 @@
 import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
 import org.apache.asterix.common.config.DatasetConfig.TransactionState;
 import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.exceptions.NoOpWarningCollector;
 import org.apache.asterix.external.api.ITypedAdapterFactory;
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.external.indexing.IndexingConstants;
@@ -64,6 +63,7 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ISecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ISecondaryIndexOperationsHelper.java
new file mode 100644
index 0000000..13ff19d
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ISecondaryIndexOperationsHelper.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.metadata.utils;
+
+import java.util.Set;
+
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
+
+public interface ISecondaryIndexOperationsHelper {
+
+    void init() throws AlgebricksException;
+
+    JobSpecification buildCreationJobSpec() throws AlgebricksException;
+
+    JobSpecification buildLoadingJobSpec() throws AlgebricksException;
+
+    JobSpecification buildCompactJobSpec() throws AlgebricksException;
+
+    JobSpecification buildDropJobSpec(Set<IndexDropOperatorDescriptor.DropOption> options) throws AlgebricksException;
+
+    IFileSplitProvider getSecondaryFileSplitProvider();
+
+    RecordDescriptor getSecondaryRecDesc();
+
+    IBinaryComparatorFactory[] getSecondaryComparatorFactories();
+
+    AlgebricksPartitionConstraint getSecondaryPartitionConstraint();
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
index ad92803..7e9cfc3 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
@@ -79,6 +79,8 @@
         } else if (index.getIndexType() == DatasetConfig.IndexType.ARRAY) {
             numSecondaryKeys = ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
                     .map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
+        } else if (index.getIndexType() == DatasetConfig.IndexType.SAMPLE) {
+            return null;
         } else {
             throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, index.getIndexType().toString());
         }
@@ -111,6 +113,8 @@
             case SINGLE_PARTITION_NGRAM_INVIX:
             case SINGLE_PARTITION_WORD_INVIX:
                 break;
+            case SAMPLE:
+                break;
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
                         index.getIndexType().toString());
@@ -120,21 +124,21 @@
 
     public static JobSpecification buildDropIndexJobSpec(Index index, MetadataProvider metadataProvider,
             Dataset dataset, SourceLocation sourceLoc) throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper =
+        ISecondaryIndexOperationsHelper secondaryIndexHelper =
                 SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
         return secondaryIndexHelper.buildDropJobSpec(EnumSet.noneOf(DropOption.class));
     }
 
     public static JobSpecification buildDropIndexJobSpec(Index index, MetadataProvider metadataProvider,
             Dataset dataset, Set<DropOption> options, SourceLocation sourceLoc) throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper =
+        ISecondaryIndexOperationsHelper secondaryIndexHelper =
                 SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
         return secondaryIndexHelper.buildDropJobSpec(options);
     }
 
     public static JobSpecification buildSecondaryIndexCreationJobSpec(Dataset dataset, Index index,
             MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper =
+        ISecondaryIndexOperationsHelper secondaryIndexHelper =
                 SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
         return secondaryIndexHelper.buildCreationJobSpec();
     }
@@ -147,8 +151,8 @@
     public static JobSpecification buildSecondaryIndexLoadingJobSpec(Dataset dataset, Index index,
             MetadataProvider metadataProvider, List<ExternalFile> files, SourceLocation sourceLoc)
             throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper;
-        if (dataset.isCorrelated()) {
+        ISecondaryIndexOperationsHelper secondaryIndexHelper;
+        if (dataset.isCorrelated() && supportsCorrelated(index.getIndexType())) { //TODO:REVISIT
             secondaryIndexHelper = SecondaryCorrelatedTreeIndexOperationsHelper.createIndexOperationsHelper(dataset,
                     index, metadataProvider, sourceLoc);
         } else {
@@ -156,14 +160,18 @@
                     metadataProvider, sourceLoc);
         }
         if (files != null) {
-            secondaryIndexHelper.setExternalFiles(files);
+            ((SecondaryIndexOperationsHelper) secondaryIndexHelper).setExternalFiles(files);
         }
         return secondaryIndexHelper.buildLoadingJobSpec();
     }
 
+    private static boolean supportsCorrelated(DatasetConfig.IndexType indexType) {
+        return indexType != DatasetConfig.IndexType.SAMPLE;
+    }
+
     public static JobSpecification buildSecondaryIndexCompactJobSpec(Dataset dataset, Index index,
             MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
-        SecondaryIndexOperationsHelper secondaryIndexHelper =
+        ISecondaryIndexOperationsHelper secondaryIndexHelper =
                 SecondaryIndexOperationsHelper.createIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
         return secondaryIndexHelper.buildCompactJobSpec();
     }
@@ -240,4 +248,9 @@
     public static boolean excludesUnknowns(Index index) {
         return !includesUnknowns(index);
     }
+
+    public static Pair<String, String> getSampleIndexNames(String datasetName) {
+        return new Pair<>(MetadataConstants.SAMPLE_INDEX_1_PREFIX + datasetName,
+                MetadataConstants.SAMPLE_INDEX_2_PREFIX + datasetName);
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
index b52cddd..0f0ea40 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
@@ -57,6 +57,11 @@
     public static final String FULL_TEXT_CONFIG_DATASET_NAME = "FullTextConfig";
     public static final String FULL_TEXT_FILTER_DATASET_NAME = "FullTextFilter";
 
+    public static final String PRIMARY_INDEX_PREFIX = "primary_idx_";
+    public static final String SAMPLE_INDEX_PREFIX = "sample_idx_";
+    public static final String SAMPLE_INDEX_1_PREFIX = SAMPLE_INDEX_PREFIX + "1_";
+    public static final String SAMPLE_INDEX_2_PREFIX = SAMPLE_INDEX_PREFIX + "2_";
+
     private MetadataConstants() {
     }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
index 7bb11e5..afd1582 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
@@ -311,6 +311,13 @@
     }
 
     @Override
+    public void analyzeDatasetBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
+            String datasetName) throws AlgebricksException {
+        lockMgr.acquireDataverseReadLock(locks, dataverseName);
+        lockMgr.acquireDatasetCreateIndexLock(locks, dataverseName, datasetName);
+    }
+
+    @Override
     public void compactBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
             String datasetName) throws AlgebricksException {
         lockMgr.acquireDataverseReadLock(locks, dataverseName);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SampleOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SampleOperationsHelper.java
new file mode 100644
index 0000000..0d3e015
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SampleOperationsHelper.java
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.metadata.utils;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.config.OptimizationConfUtil;
+import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.asterix.common.utils.StorageConstants;
+import org.apache.asterix.formats.base.IDataFormat;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.aggregates.collections.FirstElementEvalFactory;
+import org.apache.asterix.runtime.evaluators.comparisons.GreaterThanDescriptor;
+import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorDescriptor;
+import org.apache.asterix.runtime.operators.StreamStatsOperatorDescriptor;
+import org.apache.asterix.runtime.runningaggregates.std.SampleSlotRunningAggregateFunctionFactory;
+import org.apache.asterix.runtime.runningaggregates.std.TidRunningAggregateDescriptor;
+import org.apache.asterix.runtime.utils.RuntimeUtils;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
+import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
+import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
+import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
+import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.evaluators.ColumnAccessEvalFactory;
+import org.apache.hyracks.algebricks.runtime.operators.aggreg.SimpleAlgebricksAccumulatingAggregatorFactory;
+import org.apache.hyracks.algebricks.runtime.operators.aggrun.RunningAggregateRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.SinkRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
+import org.apache.hyracks.algebricks.runtime.operators.std.StreamProjectRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.std.StreamSelectRuntimeFactory;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.dataflow.std.group.AbstractAggregatorDescriptorFactory;
+import org.apache.hyracks.dataflow.std.group.sort.SortGroupByOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.api.IIndexBuilderFactory;
+import org.apache.hyracks.storage.am.common.build.IndexBuilderFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IndexCreateOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.common.IResourceFactory;
+
+/**
+ * Utility class for sampling operations.
+ * <p>
+ * The sampling method described in:
+ * "A Convenient Algorithm for Drawing a Simple Random Sample",
+ * by A. I. McLeod and D. R. Bellhouse
+ */
+public class SampleOperationsHelper implements ISecondaryIndexOperationsHelper {
+
+    public static final String DATASET_STATS_OPERATOR_NAME = "Sample.DatasetStats";
+
+    private final MetadataProvider metadataProvider;
+    private final Dataset dataset;
+    private final Index index;
+    private final SourceLocation sourceLoc;
+
+    private ARecordType itemType;
+    private ARecordType metaType;
+    private RecordDescriptor recordDesc;
+    private IBinaryComparatorFactory[] comparatorFactories;
+    private IFileSplitProvider fileSplitProvider;
+    private AlgebricksPartitionConstraint partitionConstraint;
+    private ILSMMergePolicyFactory mergePolicyFactory;
+    private Map<String, String> mergePolicyProperties;
+    private int groupbyNumFrames;
+
+    protected SampleOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) {
+        this.dataset = dataset;
+        this.index = index;
+        this.metadataProvider = metadataProvider;
+        this.sourceLoc = sourceLoc;
+    }
+
+    @Override
+    public void init() throws AlgebricksException {
+        itemType =
+                (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+        metaType = DatasetUtil.getMetaType(metadataProvider, dataset);
+        recordDesc = dataset.getPrimaryRecordDescriptor(metadataProvider);
+        comparatorFactories = dataset.getPrimaryComparatorFactories(metadataProvider, itemType, metaType);
+        groupbyNumFrames = getGroupByNumFrames(metadataProvider, sourceLoc);
+
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint =
+                metadataProvider.getSplitProviderAndConstraints(dataset, index.getIndexName());
+        fileSplitProvider = secondarySplitsAndConstraint.first;
+        partitionConstraint = secondarySplitsAndConstraint.second;
+
+        Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+                DatasetUtil.getMergePolicyFactory(dataset, metadataProvider.getMetadataTxnContext());
+        mergePolicyFactory = compactionInfo.first;
+        mergePolicyProperties = compactionInfo.second;
+    }
+
+    @Override
+    public JobSpecification buildCreationJobSpec() throws AlgebricksException {
+        JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
+        IStorageComponentProvider storageComponentProvider = metadataProvider.getStorageComponentProvider();
+        IResourceFactory resourceFactory = dataset.getResourceFactory(metadataProvider, index, itemType, metaType,
+                mergePolicyFactory, mergePolicyProperties);
+        IIndexBuilderFactory indexBuilderFactory = new IndexBuilderFactory(storageComponentProvider.getStorageManager(),
+                fileSplitProvider, resourceFactory, true);
+        IndexCreateOperatorDescriptor indexCreateOp = new IndexCreateOperatorDescriptor(spec, indexBuilderFactory);
+        indexCreateOp.setSourceLocation(sourceLoc);
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp, partitionConstraint);
+        spec.addRoot(indexCreateOp);
+        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+        return spec;
+    }
+
+    @Override
+    public JobSpecification buildLoadingJobSpec() throws AlgebricksException {
+        Index.SampleIndexDetails indexDetails = (Index.SampleIndexDetails) index.getIndexDetails();
+        int sampleCardinalityTarget = indexDetails.getSampleCardinalityTarget();
+        long sampleSeed = indexDetails.getSampleSeed();
+        IDataFormat format = metadataProvider.getDataFormat();
+        int nFields = recordDesc.getFieldCount();
+        int[] columns = new int[nFields];
+        for (int i = 0; i < nFields; i++) {
+            columns[i] = i;
+        }
+
+        JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
+        IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
+                metadataProvider.getStorageComponentProvider().getStorageManager(), fileSplitProvider);
+
+        // job spec:
+        IndexUtil.bindJobEventListener(spec, metadataProvider);
+
+        // dummy key provider ----> primary index scan
+        IOperatorDescriptor sourceOp = DatasetUtil.createDummyKeyProviderOp(spec, dataset, metadataProvider);
+        IOperatorDescriptor targetOp = DatasetUtil.createPrimaryIndexScanOp(spec, metadataProvider, dataset);
+        spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+        sourceOp = targetOp;
+
+        // primary index scan ----> stream stats op
+        targetOp = new StreamStatsOperatorDescriptor(spec, recordDesc, DATASET_STATS_OPERATOR_NAME);
+        spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+        sourceOp = targetOp;
+
+        // stream stats op ----> (running agg + select)
+        // ragg produces a slot number and a tuple counter for each tuple
+        // If the slot number is 0 then the tuple is not in the sample and is removed by subsequent select op.
+        // If the slot number is greater than 0 then the tuple is in the sample.
+        // There could be several tuples with the same slot number, the latest one wins
+        // (with the greatest tuple counter). This is accomplished by the group by below
+        BuiltinType raggSlotType = BuiltinType.AINT32;
+        BuiltinType raggCounterType = BuiltinType.AINT64;
+        int[] raggProjectColumns = new int[nFields + 2];
+        raggProjectColumns[0] = nFields;
+        raggProjectColumns[1] = nFields + 1;
+        System.arraycopy(columns, 0, raggProjectColumns, 2, nFields);
+        int[] raggAggColumns = { nFields, nFields + 1 };
+
+        ISerializerDeserializerProvider serdeProvider = format.getSerdeProvider();
+        ISerializerDeserializer[] raggSerdes = new ISerializerDeserializer[nFields + 2];
+        raggSerdes[0] = serdeProvider.getSerializerDeserializer(raggSlotType);
+        raggSerdes[1] = serdeProvider.getSerializerDeserializer(raggCounterType);
+        System.arraycopy(recordDesc.getFields(), 0, raggSerdes, 2, nFields);
+
+        ITypeTraitProvider typeTraitProvider = format.getTypeTraitProvider();
+        ITypeTraits[] raggTraits = new ITypeTraits[nFields + 2];
+        raggTraits[0] = typeTraitProvider.getTypeTrait(raggSlotType);
+        raggTraits[1] = typeTraitProvider.getTypeTrait(raggCounterType);
+        System.arraycopy(recordDesc.getTypeTraits(), 0, raggTraits, 2, nFields);
+
+        RecordDescriptor raggRecordDesc = new RecordDescriptor(raggSerdes, raggTraits);
+
+        IRunningAggregateEvaluatorFactory raggSlotEvalFactory =
+                new SampleSlotRunningAggregateFunctionFactory(sampleCardinalityTarget, sampleSeed);
+        IRunningAggregateEvaluatorFactory raggCounterEvalFactory = TidRunningAggregateDescriptor.FACTORY
+                .createFunctionDescriptor().createRunningAggregateEvaluatorFactory(new IScalarEvaluatorFactory[0]);
+        RunningAggregateRuntimeFactory raggRuntimeFactory =
+                new RunningAggregateRuntimeFactory(raggProjectColumns, raggAggColumns,
+                        new IRunningAggregateEvaluatorFactory[] { raggSlotEvalFactory, raggCounterEvalFactory });
+
+        IFunctionDescriptor gtDescriptor = GreaterThanDescriptor.FACTORY.createFunctionDescriptor();
+        gtDescriptor.setImmutableStates(raggSlotType, raggSlotType);
+        IScalarEvaluatorFactory gtFactory =
+                gtDescriptor.createEvaluatorFactory(new IScalarEvaluatorFactory[] { new ColumnAccessEvalFactory(0),
+                        format.getConstantEvalFactory(new AsterixConstantValue(new AInt32(0))) });
+        StreamSelectRuntimeFactory selectRuntimeFactory = new StreamSelectRuntimeFactory(gtFactory, null,
+                format.getBinaryBooleanInspectorFactory(), false, -1, null);
+
+        targetOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
+                new IPushRuntimeFactory[] { raggRuntimeFactory, selectRuntimeFactory },
+                new RecordDescriptor[] { raggRecordDesc, raggRecordDesc });
+        spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+        sourceOp = targetOp;
+
+        // (running agg + select) ---> group-by
+        int[] groupFields = new int[] { 0 }; // [slot]
+        int[] sortFields = new int[] { 0, 1 }; // [slot, counter]
+        OrderOperator.IOrder sortSlotOrder = OrderOperator.ASC_ORDER;
+        OrderOperator.IOrder sortCounterOrder = OrderOperator.DESC_ORDER;
+        IBinaryComparatorFactoryProvider comparatorFactoryProvider = format.getBinaryComparatorFactoryProvider();
+        IBinaryComparatorFactory[] raggCmpFactories = {
+                comparatorFactoryProvider.getBinaryComparatorFactory(raggSlotType,
+                        sortSlotOrder.getKind() == OrderOperator.IOrder.OrderKind.ASC),
+                comparatorFactoryProvider.getBinaryComparatorFactory(raggCounterType,
+                        sortCounterOrder.getKind() == OrderOperator.IOrder.OrderKind.ASC) };
+
+        INormalizedKeyComputerFactoryProvider normKeyProvider = format.getNormalizedKeyComputerFactoryProvider();
+        INormalizedKeyComputerFactory[] normKeyFactories = {
+                normKeyProvider.getNormalizedKeyComputerFactory(raggSlotType,
+                        sortSlotOrder.getKind() == OrderOperator.IOrder.OrderKind.ASC),
+                normKeyProvider.getNormalizedKeyComputerFactory(raggCounterType,
+                        sortCounterOrder.getKind() == OrderOperator.IOrder.OrderKind.ASC) };
+
+        // agg = [counter, .. original columns ..]
+        IAggregateEvaluatorFactory[] aggFactories = new IAggregateEvaluatorFactory[nFields + 1];
+        for (int i = 0; i < aggFactories.length; i++) {
+            aggFactories[i] = new FirstElementEvalFactory(
+                    new IScalarEvaluatorFactory[] { new ColumnAccessEvalFactory(1 + i) }, false, sourceLoc);
+        }
+        AbstractAggregatorDescriptorFactory aggregatorFactory =
+                new SimpleAlgebricksAccumulatingAggregatorFactory(aggFactories, groupFields);
+
+        targetOp = new SortGroupByOperatorDescriptor(spec, groupbyNumFrames, sortFields, groupFields, normKeyFactories,
+                raggCmpFactories, aggregatorFactory, aggregatorFactory, raggRecordDesc, raggRecordDesc, false);
+        spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+        sourceOp = targetOp;
+
+        // group by --> project (remove ragg fields)
+        int[] projectColumns = new int[nFields];
+        for (int i = 0; i < nFields; i++) {
+            projectColumns[i] = 2 + i;
+        }
+        StreamProjectRuntimeFactory projectRuntimeFactory = new StreamProjectRuntimeFactory(projectColumns);
+        targetOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1, new IPushRuntimeFactory[] { projectRuntimeFactory },
+                new RecordDescriptor[] { recordDesc });
+        spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+        sourceOp = targetOp;
+
+        // project ---> bulk load op
+        targetOp = createTreeIndexBulkLoadOp(spec, columns, dataflowHelperFactory,
+                StorageConstants.DEFAULT_TREE_FILL_FACTOR, sampleCardinalityTarget);
+        spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+        sourceOp = targetOp;
+
+        // bulk load op ----> sink op
+        SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+        sinkRuntimeFactory.setSourceLocation(sourceLoc);
+        targetOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0, new IPushRuntimeFactory[] { sinkRuntimeFactory },
+                new RecordDescriptor[] { recordDesc });
+        spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+
+        spec.addRoot(targetOp);
+        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+
+        return spec;
+    }
+
+    protected LSMIndexBulkLoadOperatorDescriptor createTreeIndexBulkLoadOp(JobSpecification spec,
+            int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor,
+            long numElementHint) {
+        LSMIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new LSMIndexBulkLoadOperatorDescriptor(spec,
+                recordDesc, fieldPermutation, fillFactor, false, numElementHint, true, dataflowHelperFactory, null,
+                LSMIndexBulkLoadOperatorDescriptor.BulkLoadUsage.LOAD, dataset.getDatasetId(), null);
+        treeIndexBulkLoadOp.setSourceLocation(sourceLoc);
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
+                partitionConstraint);
+        return treeIndexBulkLoadOp;
+    }
+
+    @Override
+    public JobSpecification buildDropJobSpec(Set<IndexDropOperatorDescriptor.DropOption> options)
+            throws AlgebricksException {
+        return SecondaryTreeIndexOperationsHelper.buildDropJobSpecImpl(dataset, index, options, metadataProvider,
+                sourceLoc);
+    }
+
+    @Override
+    public JobSpecification buildCompactJobSpec() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public IFileSplitProvider getSecondaryFileSplitProvider() {
+        return fileSplitProvider;
+    }
+
+    @Override
+    public RecordDescriptor getSecondaryRecDesc() {
+        return recordDesc;
+    }
+
+    @Override
+    public IBinaryComparatorFactory[] getSecondaryComparatorFactories() {
+        return comparatorFactories;
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getSecondaryPartitionConstraint() {
+        return partitionConstraint;
+    }
+
+    private static int getGroupByNumFrames(MetadataProvider metadataProvider, SourceLocation sourceLoc)
+            throws AlgebricksException {
+        return OptimizationConfUtil.getGroupByNumFrames(
+                metadataProvider.getApplicationContext().getCompilerProperties(), metadataProvider.getConfig(),
+                sourceLoc);
+    }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index 9e0d650..694b153 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -91,7 +91,7 @@
 // TODO: We should eventually have a hierarchy of classes that can create all
 // possible index job specs,
 // not just for creation.
-public abstract class SecondaryIndexOperationsHelper {
+public abstract class SecondaryIndexOperationsHelper implements ISecondaryIndexOperationsHelper {
     protected final MetadataProvider metadataProvider;
     protected final Dataset dataset;
     protected final Index index;
@@ -160,10 +160,10 @@
                 metadataProvider.getConfig(), sourceLoc);
     }
 
-    public static SecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
+    public static ISecondaryIndexOperationsHelper createIndexOperationsHelper(Dataset dataset, Index index,
             MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
 
-        SecondaryIndexOperationsHelper indexOperationsHelper;
+        ISecondaryIndexOperationsHelper indexOperationsHelper;
         switch (index.getIndexType()) {
             case ARRAY:
                 indexOperationsHelper =
@@ -182,6 +182,9 @@
                 indexOperationsHelper =
                         new SecondaryInvertedIndexOperationsHelper(dataset, index, metadataProvider, sourceLoc);
                 break;
+            case SAMPLE:
+                indexOperationsHelper = new SampleOperationsHelper(dataset, index, metadataProvider, sourceLoc);
+                break;
             default:
                 throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, sourceLoc,
                         index.getIndexType());
@@ -202,7 +205,8 @@
 
     protected abstract int getNumSecondaryKeys();
 
-    protected void init() throws AlgebricksException {
+    @Override
+    public void init() throws AlgebricksException {
         payloadSerde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
         metaSerde =
                 metaType == null ? null : SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(metaType);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
index 0373e32..e8825fc 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryTreeIndexOperationsHelper.java
@@ -72,13 +72,19 @@
 
     @Override
     public JobSpecification buildDropJobSpec(Set<DropOption> options) throws AlgebricksException {
+        return buildDropJobSpecImpl(dataset, index, options, metadataProvider, sourceLoc);
+    }
+
+    static JobSpecification buildDropJobSpecImpl(Dataset dataset, Index index, Set<DropOption> dropOptions,
+            MetadataProvider metadataProvider, SourceLocation sourceLoc) throws AlgebricksException {
         JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
                 metadataProvider.getSplitProviderAndConstraints(dataset, index.getIndexName());
         IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
                 metadataProvider.getStorageComponentProvider().getStorageManager(), splitsAndConstraint.first);
         // The index drop operation should be persistent regardless of temp datasets or permanent dataset.
-        IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec, dataflowHelperFactory, options);
+        IndexDropOperatorDescriptor btreeDrop =
+                new IndexDropOperatorDescriptor(spec, dataflowHelperFactory, dropOptions);
         btreeDrop.setSourceLocation(sourceLoc);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeDrop,
                 splitsAndConstraint.second);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ABoolean.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ABoolean.java
index 5589205..0664fec 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ABoolean.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ABoolean.java
@@ -50,7 +50,7 @@
 
     @Override
     public String toString() {
-        return Boolean.toString(bVal).toUpperCase();
+        return Boolean.toString(bVal);
     }
 
     @Override
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
index d964079..cb6c5fe 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
@@ -20,6 +20,7 @@
 
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.hyracks.util.JSONUtil;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ArrayNode;
@@ -89,7 +90,7 @@
                 if (i > 0) {
                     sb.append(", ");
                 }
-                sb.append(type.getFieldNames()[i]);
+                JSONUtil.quoteAndEscape(sb, type.getFieldNames()[i]);
                 sb.append(": ");
                 sb.append(fields[i]);
             }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/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-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java
index b787e26..6c5c0c7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/collections/FirstElementEvalFactory.java
@@ -32,14 +32,14 @@
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-class FirstElementEvalFactory implements IAggregateEvaluatorFactory {
+public class FirstElementEvalFactory implements IAggregateEvaluatorFactory {
 
     private static final long serialVersionUID = 1L;
     private final IScalarEvaluatorFactory[] args;
     private final boolean isLocal;
     private final SourceLocation sourceLoc;
 
-    FirstElementEvalFactory(IScalarEvaluatorFactory[] args, boolean isLocal, SourceLocation sourceLoc) {
+    public FirstElementEvalFactory(IScalarEvaluatorFactory[] args, boolean isLocal, SourceLocation sourceLoc) {
         this.args = args;
         this.isLocal = isLocal;
         this.sourceLoc = sourceLoc;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StreamStats.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StreamStats.java
new file mode 100644
index 0000000..268009f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StreamStats.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.operators;
+
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+
+/**
+ * Helper method to access stats produced by {@link org.apache.asterix.runtime.operators.StreamStatsOperatorDescriptor}
+ */
+public final class StreamStats {
+
+    private final long cardinality;
+
+    private final int avgTupleSize;
+
+    public StreamStats(IOperatorStats opStats) {
+        this.cardinality = opStats.getTupleCounter().get();
+        long totalTupleSize = opStats.getDiskIoCounter().get();
+        this.avgTupleSize = cardinality > 0 ? (int) (totalTupleSize / cardinality) : 0;
+    }
+
+    static void update(IOperatorStats opStats, long tupleCount, long tupleSize) {
+        opStats.getTupleCounter().update(tupleCount);
+        opStats.getDiskIoCounter().update(tupleSize);
+    }
+
+    public long getCardinality() {
+        return cardinality;
+    }
+
+    public int getAvgTupleSize() {
+        return avgTupleSize;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StreamStatsOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StreamStatsOperatorDescriptor.java
new file mode 100644
index 0000000..353401a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/StreamStatsOperatorDescriptor.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.asterix.runtime.operators;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.IStatsCollector;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+
+/**
+ * Computes total tuple count and total tuple length for all input tuples,
+ * and emits these values as operator stats.
+ */
+public final class StreamStatsOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    private final String operatorName;
+
+    public StreamStatsOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor rDesc,
+            String operatorName) {
+        super(spec, 1, 1);
+        outRecDescs[0] = rDesc;
+        this.operatorName = operatorName;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+
+        return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+
+            private FrameTupleAccessor fta;
+            private long totalTupleCount;
+            private long totalTupleLength;
+
+            @Override
+            public void open() throws HyracksDataException {
+                fta = new FrameTupleAccessor(outRecDescs[0]);
+                totalTupleCount = 0;
+                writer.open();
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                fta.reset(buffer);
+                computeStats();
+                FrameUtils.flushFrame(buffer, writer);
+            }
+
+            private void computeStats() {
+                int n = fta.getTupleCount();
+                totalTupleCount += n;
+                for (int i = 0; i < n; i++) {
+                    totalTupleLength += fta.getTupleLength(i);
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+                writer.fail();
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                IStatsCollector statsCollector = ctx.getStatsCollector();
+                if (statsCollector != null) {
+                    IOperatorStats stats = statsCollector.getOrAddOperatorStats(operatorName);
+                    StreamStats.update(stats, totalTupleCount, totalTupleLength);
+                }
+                writer.close();
+            }
+
+            @Override
+            public void flush() throws HyracksDataException {
+                writer.flush();
+            }
+
+            @Override
+            public String getDisplayName() {
+                return operatorName;
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/SampleSlotRunningAggregateFunctionFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/SampleSlotRunningAggregateFunctionFactory.java
new file mode 100644
index 0000000..a4bda44
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/SampleSlotRunningAggregateFunctionFactory.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.runningaggregates.std;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * See {@code SampleOperationsHelper} for the sampling algorithm details.
+ */
+public class SampleSlotRunningAggregateFunctionFactory implements IRunningAggregateEvaluatorFactory {
+
+    private static final long serialVersionUID = 2L;
+
+    private final int sampleCardinalityTarget;
+
+    private final long sampleSeed;
+
+    public SampleSlotRunningAggregateFunctionFactory(int sampleCardinalityTarget, long sampleSeed) {
+        this.sampleCardinalityTarget = sampleCardinalityTarget;
+        this.sampleSeed = sampleSeed;
+    }
+
+    @Override
+    public IRunningAggregateEvaluator createRunningAggregateEvaluator(IEvaluatorContext ctx)
+            throws HyracksDataException {
+
+        int sampleCardinalityTargetPerPartition = getSampleCardinalityTargetPerPartition(sampleCardinalityTarget,
+                ctx.getTaskContext().getPartitionCount());
+
+        return new IRunningAggregateEvaluator() {
+
+            private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+            private final DataOutput resultOutput = resultStorage.getDataOutput();
+            @SuppressWarnings("unchecked")
+            private final ISerializerDeserializer<AInt32> int32Serde =
+                    SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+            private final AMutableInt32 aInt32 = new AMutableInt32(0);
+
+            private final Random rnd = new Random(sampleSeed);
+            private long counter;
+
+            @Override
+            public void init() {
+                counter = 0;
+            }
+
+            @Override
+            public void step(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+                try {
+                    counter++;
+                    int outValue = evaluate();
+
+                    resultStorage.reset();
+                    aInt32.setValue(outValue);
+                    int32Serde.serialize(aInt32, resultOutput);
+                    result.set(resultStorage);
+                } catch (IOException e) {
+                    throw HyracksDataException.create(e);
+                }
+            }
+
+            private int evaluate() {
+                if (counter <= sampleCardinalityTargetPerPartition) {
+                    return (int) counter;
+                } else {
+                    long v = 1 + (long) (rnd.nextDouble() * counter);
+                    return v <= sampleCardinalityTargetPerPartition ? (int) v : 0;
+                }
+            }
+        };
+    }
+
+    private static int getSampleCardinalityTargetPerPartition(int sampleCardinalityTarget, int nPartitions) {
+        return Math.max(1, sampleCardinalityTarget / nPartitions + Math.min(sampleCardinalityTarget % nPartitions, 1));
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
index 95de235..c39db40 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
@@ -48,8 +48,7 @@
 
             @SuppressWarnings("unchecked")
             @Override
-            public IRunningAggregateEvaluator createRunningAggregateEvaluator(IEvaluatorContext ctx)
-                    throws HyracksDataException {
+            public IRunningAggregateEvaluator createRunningAggregateEvaluator(IEvaluatorContext ctx) {
 
                 return new IRunningAggregateEvaluator() {
 
@@ -57,7 +56,7 @@
                     private final ISerializerDeserializer<AInt64> serde =
                             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
                     private final AMutableInt64 m = new AMutableInt64(0);
-                    private int cnt;
+                    private long cnt;
 
                     @Override
                     public void step(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
@@ -69,7 +68,7 @@
                     }
 
                     @Override
-                    public void init() throws HyracksDataException {
+                    public void init() {
                         cnt = 1;
                     }
                 };
diff --git a/asterixdb/asterix-server/pom.xml b/asterixdb/asterix-server/pom.xml
index c5ba4cc..3963e03 100644
--- a/asterixdb/asterix-server/pom.xml
+++ b/asterixdb/asterix-server/pom.xml
@@ -353,12 +353,30 @@
             </override>
             <override>
               <gavs>
+                <gav>io.opencensus:opencensus-proto:0.2.0</gav>
+              </gavs>
+              <url>https://raw.githubusercontent.com/census-instrumentation/opencensus-proto/v0.2.0/LICENSE</url>
+            </override>
+            <override>
+              <gavs>
                 <gav>io.opencensus:opencensus-api:0.28.0</gav>
                 <gav>io.opencensus:opencensus-contrib-http-util:0.28.0</gav>
               </gavs>
               <url>https://raw.githubusercontent.com/census-instrumentation/opencensus-java/v0.28.0/LICENSE</url>
             </override>
             <override>
+              <gavs>
+                <gav>io.opencensus:opencensus-contrib-exemplar-util:0.31.0</gav>
+                <gav>io.opencensus:opencensus-contrib-grpc-metrics:0.31.0</gav>
+                <gav>io.opencensus:opencensus-contrib-resource-util:0.31.0</gav>
+                <gav>io.opencensus:opencensus-exporter-metrics-util:0.31.0</gav>
+                <gav>io.opencensus:opencensus-exporter-stats-stackdriver:0.31.0</gav>
+                <gav>io.opencensus:opencensus-impl-core:0.31.0</gav>
+                <gav>io.opencensus:opencensus-impl:0.31.0</gav>
+              </gavs>
+              <url>https://raw.githubusercontent.com/census-instrumentation/opencensus-java/v0.31.0/LICENSE</url>
+            </override>
+            <override>
               <gav>com.google.api-client:google-api-client:1.33.0</gav>
               <url>https://raw.githubusercontent.com/googleapis/google-api-java-client/v1.33.0/LICENSE</url>
             </override>
@@ -402,7 +420,20 @@
               <url>https://raw.githubusercontent.com/google/j2objc/1.3/LICENSE</url>
             </override>
             <override>
-              <gav>io.grpc:grpc-context:1.43.2</gav>
+              <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-context: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>
@@ -418,6 +449,68 @@
               <gav>com.github.luben:zstd-jni:1.4.9-1</gav>
               <url>https://raw.githubusercontent.com/luben/zstd-jni/v1.4.9-1/LICENSE</url>
             </override>
+            <override>
+              <gavs>
+                <gav>com.google.cloud.bigdataoss:gcs-connector:hadoop3-2.2.6</gav>
+                <gav>com.google.cloud.bigdataoss:gcsio:2.2.6</gav>
+                <gav>com.google.cloud.bigdataoss:util-hadoop:hadoop3-2.2.6</gav>
+                <gav>com.google.cloud.bigdataoss:util:2.2.6</gav>
+              </gavs>
+              <url>https://raw.githubusercontent.com/GoogleCloudDataproc/hadoop-connectors/v2.2.6/LICENSE</url>
+            </override>
+            <override>
+              <gavs>
+                <gav>com.google.flogger:flogger:0.7.1</gav>
+                <gav>com.google.flogger:google-extensions:0.7.1</gav>
+                <gav>com.google.flogger:flogger-system-backend:0.7.1</gav>
+              </gavs>
+              <url>https://raw.githubusercontent.com/google/flogger/flogger-0.7.1/LICENSE</url>
+            </override>
+            <override>
+              <gavs>
+                <gav>com.google.api.grpc:proto-google-cloud-monitoring-v3:1.64.0</gav>
+                <gav>com.google.api.grpc:proto-google-cloud-storage-v2:2.2.2-alpha</gav>
+                <gav>com.google.api.grpc:grpc-google-cloud-storage-v2:2.2.2-alpha</gav>
+              </gavs>
+              <url>https://raw.githubusercontent.com/googleapis/googleapis/master/LICENSE</url>
+            </override>
+            <override>
+              <gav>com.lmax:disruptor:3.4.2</gav>
+              <url>https://raw.githubusercontent.com/LMAX-Exchange/disruptor/3.4.2/LICENCE.txt</url>
+            </override>
+            <override>
+              <gav>com.google.cloud:google-cloud-core-grpc:1.82.0</gav>
+              <url>https://raw.githubusercontent.com/googleapis/java-core/v1.82.0/LICENSE</url>
+            </override>
+            <override>
+              <gav>com.google.cloud:google-cloud-monitoring:1.82.0</gav>
+              <url>https://raw.githubusercontent.com/googleapis/java-monitoring/1.82.0/LICENSE</url>
+            </override>
+            <override>
+              <gav>com.google.api-client:google-api-client-jackson2:1.32.2</gav>
+              <url>https://raw.githubusercontent.com/googleapis/google-api-java-client/v1.32.2/LICENSE</url>
+            </override>
+            <override>
+              <gav>org.conscrypt:conscrypt-openjdk-uber:2.5.1</gav>
+              <url>https://raw.githubusercontent.com/google/conscrypt/2.5.1/LICENSE</url>
+              <noticeUrl>https://raw.githubusercontent.com/google/conscrypt/2.5.1/NOTICE</noticeUrl>
+            </override>
+            <override>
+              <gav>io.perfmark:perfmark-api:0.23.0</gav>
+              <url>https://raw.githubusercontent.com/perfmark/perfmark/v0.23.0/LICENSE</url>
+              <noticeUrl>https://raw.githubusercontent.com/perfmark/perfmark/v0.23.0/NOTICE</noticeUrl>
+            </override>
+            <override>
+              <gav>com.google.api:gax-grpc:2.7.1</gav>
+              <url>https://raw.githubusercontent.com/googleapis/gax-java/v2.7.1/LICENSE</url>
+            </override>
+            <override>
+              <gavs>
+                <gav>org.bouncycastle:bcprov-jdk15on:1.60</gav>
+                <gav>org.bouncycastle:bcpkix-jdk15on:1.60</gav>
+              </gavs>
+              <url>https://raw.githubusercontent.com/bcgit/bc-java/r1rv60/LICENSE.html</url>
+            </override>
           </overrides>
           <licenses>
             <license>
@@ -466,6 +559,18 @@
                 <aliasUrl>https://raw.githubusercontent.com/googleapis/java-iam/v1.2.0/proto-google-iam-v1/LICENSE</aliasUrl>
                 <aliasUrl>https://raw.githubusercontent.com/googleapis/java-storage/v2.3.0/LICENSE</aliasUrl>
                 <aliasUrl>http://repository.jboss.org/licenses/apache-2.0.txt</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/census-instrumentation/opencensus-java/v0.31.0/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/census-instrumentation/opencensus-proto/v0.2.0/LICENSE</aliasUrl>
+                <alisUrl>https://raw.githubusercontent.com/google/conscrypt/2.5.1/LICENSE</alisUrl>
+                <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/googleapis/master/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/googleapis/java-core/v1.82.0/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/googleapis/java-monitoring/1.82.0/LICENSE</aliasUrl>
+                <aliasUrl>https://raw.githubusercontent.com/GoogleCloudDataproc/hadoop-connectors/v2.2.6/LICENSE</aliasUrl>
+                <alisUrl>https://raw.githubusercontent.com/LMAX-Exchange/disruptor/3.4.2/LICENCE.txt</alisUrl>
+                <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>
               </aliasUrls>
               <metric>1</metric>
             </license>
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index 617d075..108765c 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -94,6 +94,7 @@
     <azuredatalakejavasdk.version>12.7.2</azuredatalakejavasdk.version>
     <gcsjavasdk.version>2.3.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>
@@ -1941,6 +1942,19 @@
         <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>
+        </exclusions>
+      </dependency>
+      <!-- Hadoop GCS end -->
     </dependencies>
   </dependencyManagement>
 
diff --git a/asterixdb/src/main/appended-resources/supplemental-models.xml b/asterixdb/src/main/appended-resources/supplemental-models.xml
index 2a8e88a..b3231b9 100644
--- a/asterixdb/src/main/appended-resources/supplemental-models.xml
+++ b/asterixdb/src/main/appended-resources/supplemental-models.xml
@@ -1161,6 +1161,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>
@@ -1174,6 +1199,32 @@
     </project>
   </supplement>
 
+  <!-- io.opencensus uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>io.opencensus</groupId>
+      <artifactId>opencensus-contrib-exemplar-util</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.opencensus uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>io.opencensus</groupId>
+      <artifactId>opencensus-contrib-grpc-metrics</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
   <!-- io.opencensus uses non-fixed ALv2 with no NOTICE file -->
   <supplement>
     <project>
@@ -1187,6 +1238,316 @@
     </project>
   </supplement>
 
+  <!-- io.opencensus uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>io.opencensus</groupId>
+      <artifactId>opencensus-contrib-resource-util</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.opencensus uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>io.opencensus</groupId>
+      <artifactId>opencensus-exporter-metrics-util</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.opencensus uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>io.opencensus</groupId>
+      <artifactId>opencensus-exporter-stats-stackdriver</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.opencensus uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>io.opencensus</groupId>
+      <artifactId>opencensus-impl-core</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.opencensus uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>io.opencensus</groupId>
+      <artifactId>opencensus-impl</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.31.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.31.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.31.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.cloud.bigdataoss uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.cloud.bigdataoss</groupId>
+      <artifactId>gcs-connector</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>hadoop3-2.2.6</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>hadoop3-2.2.6</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>hadoop3-2.2.6</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.cloud.bigdataoss uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.cloud.bigdataoss</groupId>
+      <artifactId>gcsio</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>2.2.6</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>2.2.6</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>2.2.6</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.cloud.bigdataoss uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.cloud.bigdataoss</groupId>
+      <artifactId>util-hadoop</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>hadoop3-2.2.6</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>hadoop3-2.2.6</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>hadoop3-2.2.6</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.cloud.bigdataoss uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.cloud.bigdataoss</groupId>
+      <artifactId>util</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>2.2.6</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>2.2.6</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>2.2.6</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.flogger uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.flogger</groupId>
+      <artifactId>flogger</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.7.1</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.7.1</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.7.1</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.flogger uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.flogger</groupId>
+      <artifactId>google-extensions</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.7.1</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.7.1</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.7.1</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.flogger uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.flogger</groupId>
+      <artifactId>flogger-system-backend</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.7.1</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.7.1</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.7.1</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.api.grpc uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.api.grpc</groupId>
+      <artifactId>proto-google-cloud-monitoring-v3</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.64.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.64.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.64.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.api.grpc uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.api.grpc</groupId>
+      <artifactId>grpc-google-cloud-storage-v2</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>2.2.2-alpha</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>2.2.2-alpha</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>2.2.2-alpha</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.api.grpc uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.api.grpc</groupId>
+      <artifactId>proto-google-cloud-storage-v2</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>2.2.2-alpha</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>2.2.2-alpha</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>2.2.2-alpha</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.lmax uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.lmax</groupId>
+      <artifactId>disruptor</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>3.4.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>3.4.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>3.4.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.cloud uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.cloud</groupId>
+      <artifactId>google-cloud-core-grpc</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.82.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.82.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.82.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.cloud uses ALv2 with no NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.cloud</groupId>
+      <artifactId>google-cloud-monitoring</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.82.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.82.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.82.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.api-client uses ALv2 LICENSE and has a NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>com.google.api-client</groupId>
+      <artifactId>google-api-client-jackson2</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.32.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.32.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.32.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- org.conscrypt uses ALv2 LICENSE and has a NOTICE file -->
+  <supplement>
+    <project>
+      <groupId>org.conscrypt</groupId>
+      <artifactId>conscrypt-openjdk-uber</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>2.5.1</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>2.5.1</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>2.5.1</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.perfmark uses ALv2 license -->
+  <supplement>
+    <project>
+      <groupId>io.perfmark</groupId>
+      <artifactId>perfmark-api</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>0.23.0</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>0.23.0</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>0.23.0</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- org.codehaus.mojo:animal-sniffer-annotations uses MIT license -->
+  <supplement>
+    <project>
+      <groupId>org.codehaus.mojo</groupId>
+      <artifactId>animal-sniffer-annotations</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.19</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.19</license.ignoreMissingEmbeddedNotice>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.apis:google-api-services-iamcredentials has embedded ASLv2 in pom.xml -->
+  <supplement>
+    <project>
+      <groupId>com.google.apis</groupId>
+      <artifactId>google-api-services-iamcredentials</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>v1-rev20210326-1.32.1</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>v1-rev20210326-1.32.1</license.ignoreMissingEmbeddedNotice>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- com.google.apis:google-api-services-iamcredentials has embedded ASLv2 in pom.xml -->
+  <supplement>
+    <project>
+      <groupId>com.google.api</groupId>
+      <artifactId>gax-grpc</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>2.7.1</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>2.7.1</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>2.7.1</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
   <!-- com.google.cloud:google-cloud-core is non-fixed ALv2 with no NOTICE file -->
   <supplement>
     <project>
@@ -1239,7 +1600,59 @@
     </project>
   </supplement>
 
-  <!-- io.grpc:grpc-context uses non-fixed ALv2 and has no NOTICE -->
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-alts</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-api</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-auth</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-census</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
   <supplement>
     <project>
       <groupId>io.grpc</groupId>
@@ -1252,6 +1665,97 @@
     </project>
   </supplement>
 
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-core</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-grpclb</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-protobuf-lite</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-protobuf</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-services</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-stub</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <!-- io.grpc uses ALv2 -->
+  <supplement>
+    <project>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-xds</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedLicense>1.43.2</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.43.2</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.43.2</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
   <!-- com.google.protobuf has no NOTICE file -->
   <supplement>
     <project>
@@ -1388,7 +1892,7 @@
     </project>
   </supplement>
 
-  <!-- org.checkerframework:checker-compat-qual has no NOTICE file -->
+  <!-- org.checkerframework:checker-qual has no NOTICE file -->
   <supplement>
     <project>
       <groupId>org.checkerframework</groupId>
@@ -1713,4 +2217,28 @@
       </properties>
     </project>
   </supplement>
+
+  <supplement>
+    <project>
+      <groupId>org.bouncycastle</groupId>
+      <artifactId>bcpkix-jdk15on</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedNotice>1.60</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreMissingEmbeddedLicense>1.60</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreLicenseOverride>1.60</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
+
+  <supplement>
+    <project>
+      <groupId>org.bouncycastle</groupId>
+      <artifactId>bcprov-jdk15on</artifactId>
+      <properties>
+        <license.ignoreMissingEmbeddedNotice>1.60</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreMissingEmbeddedLicense>1.60</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreLicenseOverride>1.60</license.ignoreLicenseOverride>
+      </properties>
+    </project>
+  </supplement>
 </supplementalDataModels>
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_bcgit_bc-java_r1rv60_LICENSE.html.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_bcgit_bc-java_r1rv60_LICENSE.html.txt
new file mode 100644
index 0000000..b8f46c5
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_bcgit_bc-java_r1rv60_LICENSE.html.txt
@@ -0,0 +1,17 @@
+Copyright (c) 2000-2018 The Legion of the Bouncy Castle Inc. (http://www.bouncycastle.org)
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software
+and associated documentation files (the "Software"), to deal in the Software without restriction,
+including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense,
+and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so,
+subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial
+portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED,
+INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR
+PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
+OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
+DEALINGS IN THE SOFTWARE.
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_conscrypt_2.5.1_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_conscrypt_2.5.1_NOTICE.txt
new file mode 100644
index 0000000..80715a6
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_conscrypt_2.5.1_NOTICE.txt
@@ -0,0 +1,30 @@
+Copyright 2016 The Android Open Source Project
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-----------------------------------------------------------------------
+This product contains a modified portion of `Netty`, a configurable network
+stack in Java, which can be obtained at:
+
+  * LICENSE:
+    * licenses/LICENSE.netty.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * http://netty.io/
+
+This product contains a modified portion of `Apache Harmony`, modular Java runtime,
+which can be obtained at:
+
+  * LICENSE:
+    * licenses/LICENSE.harmony.txt (Apache License 2.0)
+  * HOMEPAGE:
+    * https://harmony.apache.org/
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gax-java_v2.7.1_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gax-java_v2.7.1_LICENSE.txt
new file mode 100644
index 0000000..6874140
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gax-java_v2.7.1_LICENSE.txt
@@ -0,0 +1,27 @@
+Copyright 2016, Google Inc. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+    * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.43.2_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.43.2_LICENSE.txt
deleted file mode 100644
index 7a4a3ea..0000000
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.43.2_LICENSE.txt
+++ /dev/null
@@ -1,202 +0,0 @@
-
-                                 Apache License
-                           Version 2.0, January 2004
-                        http://www.apache.org/licenses/
-
-   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-   1. Definitions.
-
-      "License" shall mean the terms and conditions for use, reproduction,
-      and distribution as defined by Sections 1 through 9 of this document.
-
-      "Licensor" shall mean the copyright owner or entity authorized by
-      the copyright owner that is granting the License.
-
-      "Legal Entity" shall mean the union of the acting entity and all
-      other entities that control, are controlled by, or are under common
-      control with that entity. For the purposes of this definition,
-      "control" means (i) the power, direct or indirect, to cause the
-      direction or management of such entity, whether by contract or
-      otherwise, or (ii) ownership of fifty percent (50%) or more of the
-      outstanding shares, or (iii) beneficial ownership of such entity.
-
-      "You" (or "Your") shall mean an individual or Legal Entity
-      exercising permissions granted by this License.
-
-      "Source" form shall mean the preferred form for making modifications,
-      including but not limited to software source code, documentation
-      source, and configuration files.
-
-      "Object" form shall mean any form resulting from mechanical
-      transformation or translation of a Source form, including but
-      not limited to compiled object code, generated documentation,
-      and conversions to other media types.
-
-      "Work" shall mean the work of authorship, whether in Source or
-      Object form, made available under the License, as indicated by a
-      copyright notice that is included in or attached to the work
-      (an example is provided in the Appendix below).
-
-      "Derivative Works" shall mean any work, whether in Source or Object
-      form, that is based on (or derived from) the Work and for which the
-      editorial revisions, annotations, elaborations, or other modifications
-      represent, as a whole, an original work of authorship. For the purposes
-      of this License, Derivative Works shall not include works that remain
-      separable from, or merely link (or bind by name) to the interfaces of,
-      the Work and Derivative Works thereof.
-
-      "Contribution" shall mean any work of authorship, including
-      the original version of the Work and any modifications or additions
-      to that Work or Derivative Works thereof, that is intentionally
-      submitted to Licensor for inclusion in the Work by the copyright owner
-      or by an individual or Legal Entity authorized to submit on behalf of
-      the copyright owner. For the purposes of this definition, "submitted"
-      means any form of electronic, verbal, or written communication sent
-      to the Licensor or its representatives, including but not limited to
-      communication on electronic mailing lists, source code control systems,
-      and issue tracking systems that are managed by, or on behalf of, the
-      Licensor for the purpose of discussing and improving the Work, but
-      excluding communication that is conspicuously marked or otherwise
-      designated in writing by the copyright owner as "Not a Contribution."
-
-      "Contributor" shall mean Licensor and any individual or Legal Entity
-      on behalf of whom a Contribution has been received by Licensor and
-      subsequently incorporated within the Work.
-
-   2. Grant of Copyright License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      copyright license to reproduce, prepare Derivative Works of,
-      publicly display, publicly perform, sublicense, and distribute the
-      Work and such Derivative Works in Source or Object form.
-
-   3. Grant of Patent License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      (except as stated in this section) patent license to make, have made,
-      use, offer to sell, sell, import, and otherwise transfer the Work,
-      where such license applies only to those patent claims licensable
-      by such Contributor that are necessarily infringed by their
-      Contribution(s) alone or by combination of their Contribution(s)
-      with the Work to which such Contribution(s) was submitted. If You
-      institute patent litigation against any entity (including a
-      cross-claim or counterclaim in a lawsuit) alleging that the Work
-      or a Contribution incorporated within the Work constitutes direct
-      or contributory patent infringement, then any patent licenses
-      granted to You under this License for that Work shall terminate
-      as of the date such litigation is filed.
-
-   4. Redistribution. You may reproduce and distribute copies of the
-      Work or Derivative Works thereof in any medium, with or without
-      modifications, and in Source or Object form, provided that You
-      meet the following conditions:
-
-      (a) You must give any other recipients of the Work or
-          Derivative Works a copy of this License; and
-
-      (b) You must cause any modified files to carry prominent notices
-          stating that You changed the files; and
-
-      (c) You must retain, in the Source form of any Derivative Works
-          that You distribute, all copyright, patent, trademark, and
-          attribution notices from the Source form of the Work,
-          excluding those notices that do not pertain to any part of
-          the Derivative Works; and
-
-      (d) If the Work includes a "NOTICE" text file as part of its
-          distribution, then any Derivative Works that You distribute must
-          include a readable copy of the attribution notices contained
-          within such NOTICE file, excluding those notices that do not
-          pertain to any part of the Derivative Works, in at least one
-          of the following places: within a NOTICE text file distributed
-          as part of the Derivative Works; within the Source form or
-          documentation, if provided along with the Derivative Works; or,
-          within a display generated by the Derivative Works, if and
-          wherever such third-party notices normally appear. The contents
-          of the NOTICE file are for informational purposes only and
-          do not modify the License. You may add Your own attribution
-          notices within Derivative Works that You distribute, alongside
-          or as an addendum to the NOTICE text from the Work, provided
-          that such additional attribution notices cannot be construed
-          as modifying the License.
-
-      You may add Your own copyright statement to Your modifications and
-      may provide additional or different license terms and conditions
-      for use, reproduction, or distribution of Your modifications, or
-      for any such Derivative Works as a whole, provided Your use,
-      reproduction, and distribution of the Work otherwise complies with
-      the conditions stated in this License.
-
-   5. Submission of Contributions. Unless You explicitly state otherwise,
-      any Contribution intentionally submitted for inclusion in the Work
-      by You to the Licensor shall be under the terms and conditions of
-      this License, without any additional terms or conditions.
-      Notwithstanding the above, nothing herein shall supersede or modify
-      the terms of any separate license agreement you may have executed
-      with Licensor regarding such Contributions.
-
-   6. Trademarks. This License does not grant permission to use the trade
-      names, trademarks, service marks, or product names of the Licensor,
-      except as required for reasonable and customary use in describing the
-      origin of the Work and reproducing the content of the NOTICE file.
-
-   7. Disclaimer of Warranty. Unless required by applicable law or
-      agreed to in writing, Licensor provides the Work (and each
-      Contributor provides its Contributions) on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
-      implied, including, without limitation, any warranties or conditions
-      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
-      PARTICULAR PURPOSE. You are solely responsible for determining the
-      appropriateness of using or redistributing the Work and assume any
-      risks associated with Your exercise of permissions under this License.
-
-   8. Limitation of Liability. In no event and under no legal theory,
-      whether in tort (including negligence), contract, or otherwise,
-      unless required by applicable law (such as deliberate and grossly
-      negligent acts) or agreed to in writing, shall any Contributor be
-      liable to You for damages, including any direct, indirect, special,
-      incidental, or consequential damages of any character arising as a
-      result of this License or out of the use or inability to use the
-      Work (including but not limited to damages for loss of goodwill,
-      work stoppage, computer failure or malfunction, or any and all
-      other commercial damages or losses), even if such Contributor
-      has been advised of the possibility of such damages.
-
-   9. Accepting Warranty or Additional Liability. While redistributing
-      the Work or Derivative Works thereof, You may choose to offer,
-      and charge a fee for, acceptance of support, warranty, indemnity,
-      or other liability obligations and/or rights consistent with this
-      License. However, in accepting such obligations, You may act only
-      on Your own behalf and on Your sole responsibility, not on behalf
-      of any other Contributor, and only if You agree to indemnify,
-      defend, and hold each Contributor harmless for any liability
-      incurred by, or claims asserted against, such Contributor by reason
-      of your accepting any such warranty or additional liability.
-
-   END OF TERMS AND CONDITIONS
-
-   APPENDIX: How to apply the Apache License to your work.
-
-      To apply the Apache License to your work, attach the following
-      boilerplate notice, with the fields enclosed by brackets "[]"
-      replaced with your own identifying information. (Don't include
-      the brackets!)  The text should be enclosed in the appropriate
-      comment syntax for the file format. We also recommend that a
-      file or class name and description of purpose be included on the
-      same "printed page" as the copyright notice for easier
-      identification within third-party archives.
-
-   Copyright [yyyy] [name of copyright owner]
-
-   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.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_perfmark_perfmark_v0.23.0_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_perfmark_perfmark_v0.23.0_NOTICE.txt
new file mode 100644
index 0000000..3ed2d12
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_perfmark_perfmark_v0.23.0_NOTICE.txt
@@ -0,0 +1,32 @@
+
+Copyright 2019 Google LLC
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+-----------------------------------------------------------------------
+
+This product contains a modified portion of 'Catapult', an open source
+Trace Event viewer for Chome, Linux, and Android applications, which can
+be obtained at:
+
+  * LICENSE:
+    * traceviewer/src/main/resources/io/perfmark/traceviewer/third_party/catapult/LICENSE (New BSD License)
+  * HOMEPAGE:
+    * https://github.com/catapult-project/catapult
+
+This product contains a modified portion of 'Polymer', a library for Web
+Components, which can be obtained at:
+  * LICENSE:
+    * traceviewer/src/main/resources/io/perfmark/traceviewer/third_party/polymer/LICENSE (New BSD License)
+  * HOMEPAGE:
+    * https://github.com/Polymer/polymer
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_typetools_checker-framework_checker-framework-2.5.3_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_typetools_checker-framework_checker-framework-2.5.3_LICENSE.txt
new file mode 100644
index 0000000..fe8c705
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_typetools_checker-framework_checker-framework-2.5.3_LICENSE.txt
@@ -0,0 +1,408 @@
+Most of the Checker Framework is licensed under the GNU General Public
+License, version 2 (GPL2), with the classpath exception.  The text of this
+license appears below.  This is the same license used for OpenJDK.
+
+A few parts of the Checker Framework have more permissive licenses.
+
+ * The annotations are licensed under the MIT License.  (The text of this
+   license appears below.)  More specifically, all the parts of the Checker
+   Framework that you might want to include with your own program use the
+   MIT License.  This is the checker-qual.jar file and all the files that
+   appear in it:  every file in a qual/ directory, plus utility files such
+   as NullnessUtil.java, RegexUtil.java, SignednessUtil.java, etc.
+   In addition, the cleanroom implementations of third-party annotations,
+   which the Checker Framework recognizes as aliases for its own
+   annotations, are licensed under the MIT License.
+
+Some external libraries that are included with the Checker Framework have
+different licenses.
+
+ * javaparser is dual licensed under the LGPL or the Apache license -- you
+   may use it under whichever one you want.  (The javaparser source code
+   contains a file with the text of the GPL, but it is not clear why, since
+   javaparser does not use the GPL.)  See file stubparser/LICENSE
+   and the source code of all its files.
+
+ * JUnit is licensed under the Common Public License v1.0 (see
+   http://www.junit.org/license), with parts (Hamcrest) licensed under the
+   BSD License (see http://hamcrest.org/JavaHamcrest/).
+
+ * plume-lib is licensed under the MIT License.
+
+The Checker Framework includes annotations for the JDK in directory
+checker/jdk/, and for some other libraries.  Each annotated library uses
+the same license as the unannotated version of the library.
+
+===========================================================================
+
+The GNU General Public License (GPL)
+
+Version 2, June 1991
+
+Copyright (C) 1989, 1991 Free Software Foundation, Inc.
+59 Temple Place, Suite 330, Boston, MA 02111-1307 USA
+
+Everyone is permitted to copy and distribute verbatim copies of this license
+document, but changing it is not allowed.
+
+Preamble
+
+The licenses for most software are designed to take away your freedom to share
+and change it.  By contrast, the GNU General Public License is intended to
+guarantee your freedom to share and change free software--to make sure the
+software is free for all its users.  This General Public License applies to
+most of the Free Software Foundation's software and to any other program whose
+authors commit to using it.  (Some other Free Software Foundation software is
+covered by the GNU Library General Public License instead.) You can apply it to
+your programs, too.
+
+When we speak of free software, we are referring to freedom, not price.  Our
+General Public Licenses are designed to make sure that you have the freedom to
+distribute copies of free software (and charge for this service if you wish),
+that you receive source code or can get it if you want it, that you can change
+the software or use pieces of it in new free programs; and that you know you
+can do these things.
+
+To protect your rights, we need to make restrictions that forbid anyone to deny
+you these rights or to ask you to surrender the rights.  These restrictions
+translate to certain responsibilities for you if you distribute copies of the
+software, or if you modify it.
+
+For example, if you distribute copies of such a program, whether gratis or for
+a fee, you must give the recipients all the rights that you have.  You must
+make sure that they, too, receive or can get the source code.  And you must
+show them these terms so they know their rights.
+
+We protect your rights with two steps: (1) copyright the software, and (2)
+offer you this license which gives you legal permission to copy, distribute
+and/or modify the software.
+
+Also, for each author's protection and ours, we want to make certain that
+everyone understands that there is no warranty for this free software.  If the
+software is modified by someone else and passed on, we want its recipients to
+know that what they have is not the original, so that any problems introduced
+by others will not reflect on the original authors' reputations.
+
+Finally, any free program is threatened constantly by software patents.  We
+wish to avoid the danger that redistributors of a free program will
+individually obtain patent licenses, in effect making the program proprietary.
+To prevent this, we have made it clear that any patent must be licensed for
+everyone's free use or not licensed at all.
+
+The precise terms and conditions for copying, distribution and modification
+follow.
+
+TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION
+
+0. This License applies to any program or other work which contains a notice
+placed by the copyright holder saying it may be distributed under the terms of
+this General Public License.  The "Program", below, refers to any such program
+or work, and a "work based on the Program" means either the Program or any
+derivative work under copyright law: that is to say, a work containing the
+Program or a portion of it, either verbatim or with modifications and/or
+translated into another language.  (Hereinafter, translation is included
+without limitation in the term "modification".) Each licensee is addressed as
+"you".
+
+Activities other than copying, distribution and modification are not covered by
+this License; they are outside its scope.  The act of running the Program is
+not restricted, and the output from the Program is covered only if its contents
+constitute a work based on the Program (independent of having been made by
+running the Program).  Whether that is true depends on what the Program does.
+
+1. You may copy and distribute verbatim copies of the Program's source code as
+you receive it, in any medium, provided that you conspicuously and
+appropriately publish on each copy an appropriate copyright notice and
+disclaimer of warranty; keep intact all the notices that refer to this License
+and to the absence of any warranty; and give any other recipients of the
+Program a copy of this License along with the Program.
+
+You may charge a fee for the physical act of transferring a copy, and you may
+at your option offer warranty protection in exchange for a fee.
+
+2. You may modify your copy or copies of the Program or any portion of it, thus
+forming a work based on the Program, and copy and distribute such modifications
+or work under the terms of Section 1 above, provided that you also meet all of
+these conditions:
+
+    a) You must cause the modified files to carry prominent notices stating
+    that you changed the files and the date of any change.
+
+    b) You must cause any work that you distribute or publish, that in whole or
+    in part contains or is derived from the Program or any part thereof, to be
+    licensed as a whole at no charge to all third parties under the terms of
+    this License.
+
+    c) If the modified program normally reads commands interactively when run,
+    you must cause it, when started running for such interactive use in the
+    most ordinary way, to print or display an announcement including an
+    appropriate copyright notice and a notice that there is no warranty (or
+    else, saying that you provide a warranty) and that users may redistribute
+    the program under these conditions, and telling the user how to view a copy
+    of this License.  (Exception: if the Program itself is interactive but does
+    not normally print such an announcement, your work based on the Program is
+    not required to print an announcement.)
+
+These requirements apply to the modified work as a whole.  If identifiable
+sections of that work are not derived from the Program, and can be reasonably
+considered independent and separate works in themselves, then this License, and
+its terms, do not apply to those sections when you distribute them as separate
+works.  But when you distribute the same sections as part of a whole which is a
+work based on the Program, the distribution of the whole must be on the terms
+of this License, whose permissions for other licensees extend to the entire
+whole, and thus to each and every part regardless of who wrote it.
+
+Thus, it is not the intent of this section to claim rights or contest your
+rights to work written entirely by you; rather, the intent is to exercise the
+right to control the distribution of derivative or collective works based on
+the Program.
+
+In addition, mere aggregation of another work not based on the Program with the
+Program (or with a work based on the Program) on a volume of a storage or
+distribution medium does not bring the other work under the scope of this
+License.
+
+3. You may copy and distribute the Program (or a work based on it, under
+Section 2) in object code or executable form under the terms of Sections 1 and
+2 above provided that you also do one of the following:
+
+    a) Accompany it with the complete corresponding machine-readable source
+    code, which must be distributed under the terms of Sections 1 and 2 above
+    on a medium customarily used for software interchange; or,
+
+    b) Accompany it with a written offer, valid for at least three years, to
+    give any third party, for a charge no more than your cost of physically
+    performing source distribution, a complete machine-readable copy of the
+    corresponding source code, to be distributed under the terms of Sections 1
+    and 2 above on a medium customarily used for software interchange; or,
+
+    c) Accompany it with the information you received as to the offer to
+    distribute corresponding source code.  (This alternative is allowed only
+    for noncommercial distribution and only if you received the program in
+    object code or executable form with such an offer, in accord with
+    Subsection b above.)
+
+The source code for a work means the preferred form of the work for making
+modifications to it.  For an executable work, complete source code means all
+the source code for all modules it contains, plus any associated interface
+definition files, plus the scripts used to control compilation and installation
+of the executable.  However, as a special exception, the source code
+distributed need not include anything that is normally distributed (in either
+source or binary form) with the major components (compiler, kernel, and so on)
+of the operating system on which the executable runs, unless that component
+itself accompanies the executable.
+
+If distribution of executable or object code is made by offering access to copy
+from a designated place, then offering equivalent access to copy the source
+code from the same place counts as distribution of the source code, even though
+third parties are not compelled to copy the source along with the object code.
+
+4. You may not copy, modify, sublicense, or distribute the Program except as
+expressly provided under this License.  Any attempt otherwise to copy, modify,
+sublicense or distribute the Program is void, and will automatically terminate
+your rights under this License.  However, parties who have received copies, or
+rights, from you under this License will not have their licenses terminated so
+long as such parties remain in full compliance.
+
+5. You are not required to accept this License, since you have not signed it.
+However, nothing else grants you permission to modify or distribute the Program
+or its derivative works.  These actions are prohibited by law if you do not
+accept this License.  Therefore, by modifying or distributing the Program (or
+any work based on the Program), you indicate your acceptance of this License to
+do so, and all its terms and conditions for copying, distributing or modifying
+the Program or works based on it.
+
+6. Each time you redistribute the Program (or any work based on the Program),
+the recipient automatically receives a license from the original licensor to
+copy, distribute or modify the Program subject to these terms and conditions.
+You may not impose any further restrictions on the recipients' exercise of the
+rights granted herein.  You are not responsible for enforcing compliance by
+third parties to this License.
+
+7. If, as a consequence of a court judgment or allegation of patent
+infringement or for any other reason (not limited to patent issues), conditions
+are imposed on you (whether by court order, agreement or otherwise) that
+contradict the conditions of this License, they do not excuse you from the
+conditions of this License.  If you cannot distribute so as to satisfy
+simultaneously your obligations under this License and any other pertinent
+obligations, then as a consequence you may not distribute the Program at all.
+For example, if a patent license would not permit royalty-free redistribution
+of the Program by all those who receive copies directly or indirectly through
+you, then the only way you could satisfy both it and this License would be to
+refrain entirely from distribution of the Program.
+
+If any portion of this section is held invalid or unenforceable under any
+particular circumstance, the balance of the section is intended to apply and
+the section as a whole is intended to apply in other circumstances.
+
+It is not the purpose of this section to induce you to infringe any patents or
+other property right claims or to contest validity of any such claims; this
+section has the sole purpose of protecting the integrity of the free software
+distribution system, which is implemented by public license practices.  Many
+people have made generous contributions to the wide range of software
+distributed through that system in reliance on consistent application of that
+system; it is up to the author/donor to decide if he or she is willing to
+distribute software through any other system and a licensee cannot impose that
+choice.
+
+This section is intended to make thoroughly clear what is believed to be a
+consequence of the rest of this License.
+
+8. If the distribution and/or use of the Program is restricted in certain
+countries either by patents or by copyrighted interfaces, the original
+copyright holder who places the Program under this License may add an explicit
+geographical distribution limitation excluding those countries, so that
+distribution is permitted only in or among countries not thus excluded.  In
+such case, this License incorporates the limitation as if written in the body
+of this License.
+
+9. The Free Software Foundation may publish revised and/or new versions of the
+General Public License from time to time.  Such new versions will be similar in
+spirit to the present version, but may differ in detail to address new problems
+or concerns.
+
+Each version is given a distinguishing version number.  If the Program
+specifies a version number of this License which applies to it and "any later
+version", you have the option of following the terms and conditions either of
+that version or of any later version published by the Free Software Foundation.
+If the Program does not specify a version number of this License, you may
+choose any version ever published by the Free Software Foundation.
+
+10. If you wish to incorporate parts of the Program into other free programs
+whose distribution conditions are different, write to the author to ask for
+permission.  For software which is copyrighted by the Free Software Foundation,
+write to the Free Software Foundation; we sometimes make exceptions for this.
+Our decision will be guided by the two goals of preserving the free status of
+all derivatives of our free software and of promoting the sharing and reuse of
+software generally.
+
+NO WARRANTY
+
+11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY FOR
+THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW.  EXCEPT WHEN OTHERWISE
+STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES PROVIDE THE
+PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED,
+INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS FOR A PARTICULAR PURPOSE.  THE ENTIRE RISK AS TO THE QUALITY AND
+PERFORMANCE OF THE PROGRAM IS WITH YOU.  SHOULD THE PROGRAM PROVE DEFECTIVE,
+YOU ASSUME THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION.
+
+12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING WILL
+ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR REDISTRIBUTE THE
+PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY
+GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR
+INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA
+BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A
+FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER
+OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+END OF TERMS AND CONDITIONS
+
+How to Apply These Terms to Your New Programs
+
+If you develop a new program, and you want it to be of the greatest possible
+use to the public, the best way to achieve this is to make it free software
+which everyone can redistribute and change under these terms.
+
+To do so, attach the following notices to the program.  It is safest to attach
+them to the start of each source file to most effectively convey the exclusion
+of warranty; and each file should have at least the "copyright" line and a
+pointer to where the full notice is found.
+
+    One line to give the program's name and a brief idea of what it does.
+
+    Copyright (C) <year> <name of author>
+
+    This program is free software; you can redistribute it and/or modify it
+    under the terms of the GNU General Public License as published by the Free
+    Software Foundation; either version 2 of the License, or (at your option)
+    any later version.
+
+    This program is distributed in the hope that it will be useful, but WITHOUT
+    ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+    FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License for
+    more details.
+
+    You should have received a copy of the GNU General Public License along
+    with this program; if not, write to the Free Software Foundation, Inc., 59
+    Temple Place, Suite 330, Boston, MA 02111-1307 USA
+
+Also add information on how to contact you by electronic and paper mail.
+
+If the program is interactive, make it output a short notice like this when it
+starts in an interactive mode:
+
+    Gnomovision version 69, Copyright (C) year name of author Gnomovision comes
+    with ABSOLUTELY NO WARRANTY; for details type 'show w'.  This is free
+    software, and you are welcome to redistribute it under certain conditions;
+    type 'show c' for details.
+
+The hypothetical commands 'show w' and 'show c' should show the appropriate
+parts of the General Public License.  Of course, the commands you use may be
+called something other than 'show w' and 'show c'; they could even be
+mouse-clicks or menu items--whatever suits your program.
+
+You should also get your employer (if you work as a programmer) or your school,
+if any, to sign a "copyright disclaimer" for the program, if necessary.  Here
+is a sample; alter the names:
+
+    Yoyodyne, Inc., hereby disclaims all copyright interest in the program
+    'Gnomovision' (which makes passes at compilers) written by James Hacker.
+
+    signature of Ty Coon, 1 April 1989
+
+    Ty Coon, President of Vice
+
+This General Public License does not permit incorporating your program into
+proprietary programs.  If your program is a subroutine library, you may
+consider it more useful to permit linking proprietary applications with the
+library.  If this is what you want to do, use the GNU Library General Public
+License instead of this License.
+
+
+"CLASSPATH" EXCEPTION TO THE GPL
+
+Certain source files distributed by Oracle America and/or its affiliates are
+subject to the following clarification and special exception to the GPL, but
+only where Oracle has expressly included in the particular source file's header
+the words "Oracle designates this particular file as subject to the "Classpath"
+exception as provided by Oracle in the LICENSE file that accompanied this code."
+
+    Linking this library statically or dynamically with other modules is making
+    a combined work based on this library.  Thus, the terms and conditions of
+    the GNU General Public License cover the whole combination.
+
+    As a special exception, the copyright holders of this library give you
+    permission to link this library with independent modules to produce an
+    executable, regardless of the license terms of these independent modules,
+    and to copy and distribute the resulting executable under terms of your
+    choice, provided that you also meet, for each linked independent module,
+    the terms and conditions of the license of that module.  An independent
+    module is a module which is not derived from or based on this library.  If
+    you modify this library, you may extend this exception to your version of
+    the library, but you are not obligated to do so.  If you do not wish to do
+    so, delete this exception statement from your version.
+
+===========================================================================
+
+MIT License:
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+===========================================================================
\ No newline at end of file
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml b/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
index fbe9dbc..f557d7a 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
@@ -59,6 +59,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>algebricks-runtime</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
       <version>${project.version}</version>
     </dependency>
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
index c22d54d..3a2fe3b 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
@@ -19,6 +19,8 @@
 package org.apache.hyracks.algebricks.compiler.api;
 
 import java.util.List;
+import java.util.function.Function;
+import java.util.function.Supplier;
 
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -32,7 +34,9 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.IPartialAggregationTypeComputer;
 import org.apache.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.algebricks.data.IAWriterFactory;
 import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import org.apache.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
@@ -40,6 +44,7 @@
 import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
 import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import org.apache.hyracks.algebricks.data.IPrinterFactoryProvider;
+import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
 import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
 import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
 import org.apache.hyracks.algebricks.data.IUnnestingPositionWriterFactory;
@@ -49,8 +54,9 @@
 
 public abstract class AbstractCompilerFactoryBuilder {
 
-    protected List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites;
-    protected List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites;
+    protected Supplier<List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> logicalRewrites;
+    protected Function<IRuleSetKind, List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> logicalRewritesByKind;
+    protected Supplier<List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> physicalRewrites;
     protected ITypeTraitProvider typeTraitProvider;
     protected ISerializerDeserializerProvider serializerDeserializerProvider;
     protected IBinaryHashFunctionFactoryProvider hashFunctionFactoryProvider;
@@ -59,6 +65,8 @@
     protected IBinaryBooleanInspectorFactory binaryBooleanInspectorFactory;
     protected IBinaryIntegerInspectorFactory binaryIntegerInspectorFactory;
     protected IPrinterFactoryProvider printerProvider;
+    protected IAWriterFactory writerFactory;
+    protected IResultSerializerFactoryProvider resultSerializerFactoryProvider;
     protected IPredicateEvaluatorFactoryProvider predEvaluatorFactoryProvider;
     protected IExpressionRuntimeProvider expressionRuntimeProvider;
     protected IExpressionTypeComputer expressionTypeComputer;
@@ -78,11 +86,18 @@
 
     public abstract ICompilerFactory create();
 
-    public void setLogicalRewrites(List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites) {
+    public void setLogicalRewrites(
+            Supplier<List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> logicalRewrites) {
         this.logicalRewrites = logicalRewrites;
     }
 
-    public void setPhysicalRewrites(List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites) {
+    public void setLogicalRewritesByKind(
+            Function<IRuleSetKind, List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> logicalRewritesByKind) {
+        this.logicalRewritesByKind = logicalRewritesByKind;
+    }
+
+    public void setPhysicalRewrites(
+            Supplier<List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>> physicalRewrites) {
         this.physicalRewrites = physicalRewrites;
     }
 
@@ -158,6 +173,22 @@
         return printerProvider;
     }
 
+    public void setWriterFactory(IAWriterFactory writerFactory) {
+        this.writerFactory = writerFactory;
+    }
+
+    public IAWriterFactory getWriterFactory() {
+        return writerFactory;
+    }
+
+    public void setResultSerializerFactoryProvider(IResultSerializerFactoryProvider resultSerializerFactoryProvider) {
+        this.resultSerializerFactoryProvider = resultSerializerFactoryProvider;
+    }
+
+    public IResultSerializerFactoryProvider getResultSerializerFactoryProvider() {
+        return resultSerializerFactoryProvider;
+    }
+
     public void setExpressionRuntimeProvider(IExpressionRuntimeProvider expressionRuntimeProvider) {
         this.expressionRuntimeProvider = expressionRuntimeProvider;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
index 891980f..e35a539 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
@@ -18,8 +18,11 @@
  */
 package org.apache.hyracks.algebricks.compiler.api;
 
+import java.util.List;
+
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IConflictingTypeResolver;
@@ -33,10 +36,15 @@
 import org.apache.hyracks.algebricks.core.config.AlgebricksConfig;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.algebricks.core.jobgen.impl.PlanCompiler;
+import org.apache.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
 import org.apache.hyracks.algebricks.core.rewriter.base.AlgebricksOptimizationContext;
 import org.apache.hyracks.algebricks.core.rewriter.base.HeuristicOptimizer;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.algebricks.data.IAWriterFactory;
+import org.apache.hyracks.algebricks.runtime.writers.SerializedDataWriterFactory;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 import org.apache.hyracks.api.job.IJobletEventListenerFactory;
 import org.apache.hyracks.api.job.JobSpecification;
@@ -58,14 +66,19 @@
                 IConflictingTypeResolver conflictingTypeResolver, PhysicalOptimizationConfig physicalOptimizationConfig,
                 AlgebricksPartitionConstraint clusterLocations, IWarningCollector warningCollector) {
             IPlanPrettyPrinter prettyPrinter = PlanPrettyPrinter.createStringPlanPrettyPrinter();
-            return new AlgebricksOptimizationContext(varCounter, expressionEvalSizeComputer,
+            return new AlgebricksOptimizationContext(this, varCounter, expressionEvalSizeComputer,
                     mergeAggregationExpressionFactory, expressionTypeComputer, missableTypeComputer,
                     conflictingTypeResolver, physicalOptimizationConfig, clusterLocations, prettyPrinter,
                     warningCollector);
         }
+
+        @Override
+        public IOptimizationContext cloneOptimizationContext(IOptimizationContext oc) {
+            return new AlgebricksOptimizationContext((AlgebricksOptimizationContext) oc);
+        }
     }
 
-    private IOptimizationContextFactory optCtxFactory;
+    private final IOptimizationContextFactory optCtxFactory;
 
     public HeuristicCompilerFactoryBuilder() {
         this.optCtxFactory = DefaultOptimizationContextFactory.INSTANCE;
@@ -77,42 +90,85 @@
 
     @Override
     public ICompilerFactory create() {
-        return new ICompilerFactory() {
-            @Override
-            public ICompiler createCompiler(final ILogicalPlan plan, final IMetadataProvider<?, ?> metadata,
-                    int varCounter) {
-                final IOptimizationContext oc = optCtxFactory.createOptimizationContext(varCounter,
-                        expressionEvalSizeComputer, mergeAggregationExpressionFactory, expressionTypeComputer,
-                        missableTypeComputer, conflictingTypeResolver, physicalOptimizationConfig, clusterLocations,
-                        warningCollector);
-                oc.setMetadataDeclarations(metadata);
-                final HeuristicOptimizer opt = new HeuristicOptimizer(plan, logicalRewrites, physicalRewrites, oc);
-                return new ICompiler() {
-
-                    @Override
-                    public void optimize() throws AlgebricksException {
-                        opt.optimize();
-                    }
-
-                    @Override
-                    public JobSpecification createJob(Object appContext,
-                            IJobletEventListenerFactory jobEventListenerFactory) throws AlgebricksException {
-                        AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Starting Job Generation.\n");
-                        JobGenContext context = new JobGenContext(null, metadata, appContext,
-                                serializerDeserializerProvider, hashFunctionFactoryProvider, hashFunctionFamilyProvider,
-                                comparatorFactoryProvider, typeTraitProvider, binaryBooleanInspectorFactory,
-                                binaryIntegerInspectorFactory, printerProvider, missingWriterFactory, nullWriterFactory,
-                                unnestingPositionWriterFactory, normalizedKeyComputerFactoryProvider,
-                                expressionRuntimeProvider, expressionTypeComputer, oc, expressionEvalSizeComputer,
-                                partialAggregationTypeComputer, predEvaluatorFactoryProvider,
-                                physicalOptimizationConfig.getFrameSize(), clusterLocations, warningCollector,
-                                maxWarnings, physicalOptimizationConfig);
-                        PlanCompiler pc = new PlanCompiler(context);
-                        return pc.compilePlan(plan, jobEventListenerFactory);
-                    }
-                };
-            }
-        };
+        return new CompilerFactoryImpl();
     }
 
+    private class CompilerFactoryImpl implements ICompilerFactory {
+        @Override
+        public ICompiler createCompiler(ILogicalPlan plan, IMetadataProvider<?, ?> metadata, int varCounter) {
+            IOptimizationContext optContext =
+                    optCtxFactory.createOptimizationContext(varCounter, expressionEvalSizeComputer,
+                            mergeAggregationExpressionFactory, expressionTypeComputer, missableTypeComputer,
+                            conflictingTypeResolver, physicalOptimizationConfig, clusterLocations, warningCollector);
+            optContext.setMetadataDeclarations(metadata);
+            optContext.setCompilerFactory(this);
+            return new CompilerImpl(this, plan, optContext, logicalRewrites.get(), physicalRewrites.get(),
+                    writerFactory);
+        }
+
+        @Override
+        public ICompiler createCompiler(ILogicalPlan plan, IOptimizationContext newOptContext,
+                IRuleSetKind ruleSetKind) {
+            if (newOptContext.getCompilerFactory() != this) {
+                throw new IllegalStateException();
+            }
+            return new CompilerImpl(this, plan, newOptContext, logicalRewritesByKind.apply(ruleSetKind),
+                    physicalRewrites.get(), SerializedDataWriterFactory.WITHOUT_RECORD_DESCRIPTOR);
+        }
+
+        private PlanCompiler createPlanCompiler(IOptimizationContext oc, Object appContext,
+                IAWriterFactory writerFactory) {
+            JobGenContext context = new JobGenContext(null, oc.getMetadataProvider(), appContext,
+                    serializerDeserializerProvider, hashFunctionFactoryProvider, hashFunctionFamilyProvider,
+                    comparatorFactoryProvider, typeTraitProvider, binaryBooleanInspectorFactory,
+                    binaryIntegerInspectorFactory, printerProvider, writerFactory, resultSerializerFactoryProvider,
+                    missingWriterFactory, nullWriterFactory, unnestingPositionWriterFactory,
+                    normalizedKeyComputerFactoryProvider, expressionRuntimeProvider, expressionTypeComputer, oc,
+                    expressionEvalSizeComputer, partialAggregationTypeComputer, predEvaluatorFactoryProvider,
+                    physicalOptimizationConfig.getFrameSize(), clusterLocations, warningCollector, maxWarnings,
+                    physicalOptimizationConfig);
+            return new PlanCompiler(context);
+        }
+    }
+
+    private static class CompilerImpl implements ICompiler {
+
+        private final CompilerFactoryImpl factory;
+
+        private final ILogicalPlan plan;
+
+        private final IOptimizationContext oc;
+
+        private final List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites;
+
+        private final List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites;
+
+        private final IAWriterFactory writerFactory;
+
+        private CompilerImpl(CompilerFactoryImpl factory, ILogicalPlan plan, IOptimizationContext oc,
+                List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites,
+                List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites,
+                IAWriterFactory writerFactory) {
+            this.factory = factory;
+            this.plan = plan;
+            this.oc = oc;
+            this.logicalRewrites = logicalRewrites;
+            this.physicalRewrites = physicalRewrites;
+            this.writerFactory = writerFactory;
+        }
+
+        @Override
+        public void optimize() throws AlgebricksException {
+            HeuristicOptimizer opt = new HeuristicOptimizer(plan, logicalRewrites, physicalRewrites, oc);
+            opt.optimize();
+        }
+
+        @Override
+        public JobSpecification createJob(Object appContext, IJobletEventListenerFactory jobEventListenerFactory)
+                throws AlgebricksException {
+            AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Starting Job Generation.\n");
+            PlanCompiler pc = factory.createPlanCompiler(oc, appContext, writerFactory);
+            return pc.compilePlan(plan, jobEventListenerFactory);
+        }
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompilerFactory.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompilerFactory.java
index 7c138ea..07a8034 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompilerFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/ICompilerFactory.java
@@ -19,8 +19,12 @@
 package org.apache.hyracks.algebricks.compiler.api;
 
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import org.apache.hyracks.algebricks.core.rewriter.base.IRuleSetKind;
 
 public interface ICompilerFactory {
     ICompiler createCompiler(ILogicalPlan plan, IMetadataProvider<?, ?> metadata, int varCounter);
+
+    ICompiler createCompiler(ILogicalPlan plan, IOptimizationContext newOptContext, IRuleSetKind ruleSetKind);
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/Counter.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/Counter.java
index 7f2d3c8..3982171 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/Counter.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/Counter.java
@@ -32,10 +32,18 @@
         return counter;
     }
 
+    public int getAndInc() {
+        return counter++;
+    }
+
     public void inc() {
         ++counter;
     }
 
+    public int incAndGet() {
+        return ++counter;
+    }
+
     public void set(int newStart) {
         counter = newStart;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
index 166ab9a..69ec210 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
@@ -33,6 +33,7 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 
@@ -93,4 +94,10 @@
     public PlanStructureVerifier getPlanStructureVerifier();
 
     public PlanStabilityVerifier getPlanStabilityVerifier();
+
+    void setCompilerFactory(Object factory);
+
+    Object getCompilerFactory();
+
+    IOptimizationContextFactory getOptimizationContextFactory();
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
index e2a68da..6f0cbba 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
@@ -55,7 +55,7 @@
 
         @Override
         public String toString() {
-            return "TRUE";
+            return "true";
         }
 
         @Override
@@ -92,7 +92,7 @@
 
         @Override
         public String toString() {
-            return "FALSE";
+            return "false";
         }
 
         @Override
@@ -129,7 +129,7 @@
 
         @Override
         public String toString() {
-            return "NULL";
+            return "null";
         }
 
         @Override
@@ -166,7 +166,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/metadata/IMetadataProvider.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
index 77fdd74..d350789 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
@@ -31,7 +31,9 @@
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.data.IAWriterFactory;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
 import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
@@ -56,11 +58,12 @@
             IProjectionInfo<?> projectionInfo) throws AlgebricksException;
 
     public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteFileRuntime(IDataSink sink,
-            int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc)
-            throws AlgebricksException;
+            int[] printColumns, IPrinterFactory[] printerFactories, IAWriterFactory writerFactory,
+            RecordDescriptor inputDesc) throws AlgebricksException;
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink,
-            int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc,
+            int[] printColumns, IPrinterFactory[] printerFactories, IAWriterFactory writerFactory,
+            IResultSerializerFactoryProvider resultSerializerFactoryProvider, RecordDescriptor inputDesc,
             IResultMetadata metadata, JobSpecification spec) throws AlgebricksException;
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getWriteResultRuntime(IDataSource<S> dataSource,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
index 5fc69b2..56e2e22 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
@@ -25,10 +25,10 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 
 public abstract class AbstractUnnestMapOperator extends AbstractUnnestOperator {
 
-    protected final Mutable<ILogicalExpression> expression;
     protected final List<Object> variableTypes;
     protected boolean propagateInput;
     protected List<Mutable<ILogicalExpression>> additionalFilteringExpressions;
@@ -42,7 +42,6 @@
     public AbstractUnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
             List<Object> variableTypes, boolean propagateInput) {
         super(variables, expression);
-        this.expression = expression;
         this.variableTypes = variableTypes;
         this.propagateInput = propagateInput;
         this.propagateIndexFilter = false;
@@ -90,6 +89,18 @@
         };
     }
 
+    @Override
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+        boolean changed = super.acceptExpressionTransform(visitor);
+
+        if (additionalFilteringExpressions != null) {
+            for (Mutable<ILogicalExpression> filterExpr : additionalFilteringExpressions) {
+                changed |= visitor.transform(filterExpr);
+            }
+        }
+        return changed;
+    }
+
     public boolean propagatesInput() {
         return propagateInput;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
index bb18014..9f73113 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
@@ -57,7 +57,7 @@
     }
 
     public DataSourceScanOperator(List<LogicalVariable> variables, IDataSource<?> dataSource,
-            Mutable<ILogicalExpression> selectCondition, long outputLimit, IProjectionInfo projectionInfo) {
+            Mutable<ILogicalExpression> selectCondition, long outputLimit, IProjectionInfo<?> projectionInfo) {
         super(variables, dataSource);
         projectVars = new ArrayList<>();
         this.selectCondition = selectCondition;
@@ -77,7 +77,13 @@
 
     @Override
     public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
-        return false;
+        boolean changed = selectCondition != null && visitor.transform(selectCondition);
+        if (additionalFilteringExpressions != null) {
+            for (Mutable<ILogicalExpression> filterExpr : additionalFilteringExpressions) {
+                changed |= visitor.transform(filterExpr);
+            }
+        }
+        return changed;
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
index c4bcc52..f8d07b8 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
 import org.apache.hyracks.algebricks.core.algebra.typing.NonPropagatingTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 
 public class UnnestMapOperator extends AbstractUnnestMapOperator {
@@ -61,6 +62,12 @@
         return visitor.visitUnnestMapOperator(this, arg);
     }
 
+    @Override
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+        boolean changed = super.acceptExpressionTransform(visitor);
+        return selectCondition != null && visitor.transform(selectCondition) || changed;
+    }
+
     // When propagateInput is true,
     // this operator propagates all input variables.
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
index 138cff8..1f7c16c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
@@ -102,8 +102,9 @@
         IPrinterFactory[] pf =
                 JobGenHelper.mkPrinterFactories(inputSchemas[0], context.getTypeEnvironment(op), context, columns);
 
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = mp.getResultHandleRuntime(
-                resultOp.getDataSink(), columns, pf, inputDesc, resultOp.getResultMetadata(), spec);
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints =
+                mp.getResultHandleRuntime(resultOp.getDataSink(), columns, pf, context.getWriterFactory(),
+                        context.getResultSerializerFactoryProvider(), inputDesc, resultOp.getResultMetadata(), spec);
         IOperatorDescriptor opDesc = runtimeAndConstraints.first;
         opDesc.setSourceLocation(resultOp.getSourceLocation());
         builder.contributeHyracksOperator(resultOp, opDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
index 083e4d3..2bd78b1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
@@ -27,7 +27,6 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
-import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
 import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
@@ -43,7 +42,9 @@
     @Override
     public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
         AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
-        deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+        if (op2.getDeliveredPhysicalProperties() != null) {
+            deliveredProperties = op2.getDeliveredPhysicalProperties().clone();
+        }
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
index 3521a27..07c798f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
@@ -101,7 +101,7 @@
         IMetadataProvider<?, ?> mp = context.getMetadataProvider();
 
         Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> runtimeAndConstraints =
-                mp.getWriteFileRuntime(write.getDataSink(), columns, pf, inputDesc);
+                mp.getWriteFileRuntime(write.getDataSink(), columns, pf, context.getWriterFactory(), inputDesc);
         IPushRuntimeFactory runtime = runtimeAndConstraints.first;
         runtime.setSourceLocation(write.getSourceLocation());
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
index 86c6066..b626ae6 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
@@ -37,4 +37,5 @@
     public static final boolean ARRAY_INDEX_DEFAULT = true;
     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/JobGenContext.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
index 7c7d5a8..471380c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
@@ -35,6 +35,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.algebricks.data.IAWriterFactory;
 import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import org.apache.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
@@ -42,6 +43,7 @@
 import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
 import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import org.apache.hyracks.algebricks.data.IPrinterFactoryProvider;
+import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
 import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
 import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
 import org.apache.hyracks.algebricks.data.IUnnestingPositionWriterFactory;
@@ -57,6 +59,8 @@
     private final IBinaryHashFunctionFamilyProvider hashFunctionFamilyProvider;
     private final IBinaryComparatorFactoryProvider comparatorFactoryProvider;
     private final IPrinterFactoryProvider printerFactoryProvider;
+    private final IAWriterFactory writerFactory;
+    private final IResultSerializerFactoryProvider resultSerializerFactoryProvider;
     private final ITypeTraitProvider typeTraitProvider;
     private final IMetadataProvider<?, ?> metadataProvider;
     private final IMissingWriterFactory missingWriterFactory;
@@ -86,6 +90,7 @@
             IBinaryComparatorFactoryProvider comparatorFactoryProvider, ITypeTraitProvider typeTraitProvider,
             IBinaryBooleanInspectorFactory booleanInspectorFactory,
             IBinaryIntegerInspectorFactory integerInspectorFactory, IPrinterFactoryProvider printerFactoryProvider,
+            IAWriterFactory writerFactory, IResultSerializerFactoryProvider resultSerializerFactoryProvider,
             IMissingWriterFactory missingWriterFactory, IMissingWriterFactory nullWriterFactory,
             IUnnestingPositionWriterFactory unnestingPositionWriterFactory,
             INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider,
@@ -106,6 +111,8 @@
         this.booleanInspectorFactory = booleanInspectorFactory;
         this.integerInspectorFactory = integerInspectorFactory;
         this.printerFactoryProvider = printerFactoryProvider;
+        this.writerFactory = writerFactory;
+        this.resultSerializerFactoryProvider = resultSerializerFactoryProvider;
         this.clusterLocations = clusterLocations;
         this.normalizedKeyComputerFactoryProvider = normalizedKeyComputerFactoryProvider;
         this.missingWriterFactory = missingWriterFactory;
@@ -172,6 +179,14 @@
         return printerFactoryProvider;
     }
 
+    public IAWriterFactory getWriterFactory() {
+        return writerFactory;
+    }
+
+    public IResultSerializerFactoryProvider getResultSerializerFactoryProvider() {
+        return resultSerializerFactoryProvider;
+    }
+
     public IPredicateEvaluatorFactoryProvider getPredicateEvaluatorFactoryProvider() {
         return predEvaluatorFactoryProvider;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
index d1fd247..bee842c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
@@ -46,45 +46,29 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.ILogicalPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
 
 /**
  * The Algebricks default implementation for IOptimizationContext.
  */
-@SuppressWarnings({ "unchecked", "rawtypes" })
+@SuppressWarnings({ "rawtypes" })
 public class AlgebricksOptimizationContext implements IOptimizationContext {
 
-    private int varCounter;
+    private final IOptimizationContextFactory optContextFactory;
     private final IExpressionEvalSizeComputer expressionEvalSizeComputer;
     private final IMergeAggregationExpressionFactory mergeAggregationExpressionFactory;
     private final PhysicalOptimizationConfig physicalOptimizationConfig;
-    private final IVariableEvalSizeEnvironment varEvalSizeEnv = new IVariableEvalSizeEnvironment() {
 
-        Map<LogicalVariable, Integer> varSizeMap = new HashMap<>();
-
-        @Override
-        public void setVariableEvalSize(LogicalVariable var, int size) {
-            varSizeMap.put(var, size);
-        }
-
-        @Override
-        public int getVariableEvalSize(LogicalVariable var) {
-            return varSizeMap.get(var);
-        }
-    };
-
-    private Map<ILogicalOperator, IVariableTypeEnvironment> typeEnvMap = new HashMap<>();
-
-    private Map<ILogicalOperator, HashSet<ILogicalOperator>> alreadyCompared = new HashMap<>();
-    private Map<IAlgebraicRewriteRule, HashSet<ILogicalOperator>> dontApply = new HashMap<>();
-    private Map<LogicalVariable, FunctionalDependency> varToPrimaryKey = new HashMap<>();
-
-    private IMetadataProvider metadataProvider;
-    private HashSet<LogicalVariable> notToBeInlinedVars = new HashSet<>();
-
+    private final VariableEvalSizeEnvironmentImpl varEvalSizeEnv = new VariableEvalSizeEnvironmentImpl();
+    private final Map<ILogicalOperator, IVariableTypeEnvironment> typeEnvMap = new HashMap<>();
+    private final Map<ILogicalOperator, HashSet<ILogicalOperator>> alreadyCompared = new HashMap<>();
+    private final Map<IAlgebraicRewriteRule, HashSet<ILogicalOperator>> dontApply = new HashMap<>();
+    private final Map<LogicalVariable, FunctionalDependency> varToPrimaryKey = new HashMap<>();
+    private final HashSet<LogicalVariable> notToBeInlinedVars = new HashSet<>();
     protected final Map<ILogicalOperator, List<FunctionalDependency>> fdGlobalMap = new HashMap<>();
     protected final Map<ILogicalOperator, Map<LogicalVariable, EquivalenceClass>> eqClassGlobalMap = new HashMap<>();
-
     protected final Map<ILogicalOperator, ILogicalPropertiesVector> logicalProps = new HashMap<>();
+
     private final IExpressionTypeComputer expressionTypeComputer;
     private final IMissableTypeComputer nullableTypeComputer;
     private final INodeDomain defaultNodeDomain;
@@ -94,12 +78,18 @@
     private final PlanStructureVerifier planStructureVerifier;
     private final PlanStabilityVerifier planStabilityVerifier;
 
-    public AlgebricksOptimizationContext(int varCounter, IExpressionEvalSizeComputer expressionEvalSizeComputer,
+    private int varCounter;
+    private IMetadataProvider metadataProvider;
+    private Object compilerFactory;
+
+    public AlgebricksOptimizationContext(IOptimizationContextFactory optContextFactory, int varCounter,
+            IExpressionEvalSizeComputer expressionEvalSizeComputer,
             IMergeAggregationExpressionFactory mergeAggregationExpressionFactory,
             IExpressionTypeComputer expressionTypeComputer, IMissableTypeComputer nullableTypeComputer,
             IConflictingTypeResolver conflictingTypeResovler, PhysicalOptimizationConfig physicalOptimizationConfig,
             AlgebricksPartitionConstraint clusterLocations, IPlanPrettyPrinter prettyPrinter,
             IWarningCollector warningCollector) {
+        this.optContextFactory = optContextFactory;
         this.varCounter = varCounter;
         this.expressionEvalSizeComputer = expressionEvalSizeComputer;
         this.mergeAggregationExpressionFactory = mergeAggregationExpressionFactory;
@@ -115,6 +105,35 @@
         this.planStabilityVerifier = isSanityCheckEnabled ? new PlanStabilityVerifier(prettyPrinter) : null;
     }
 
+    public AlgebricksOptimizationContext(AlgebricksOptimizationContext from) {
+        optContextFactory = from.optContextFactory;
+        varCounter = from.varCounter;
+        expressionEvalSizeComputer = from.expressionEvalSizeComputer;
+        mergeAggregationExpressionFactory = from.mergeAggregationExpressionFactory;
+        expressionTypeComputer = from.expressionTypeComputer;
+        nullableTypeComputer = from.nullableTypeComputer;
+        physicalOptimizationConfig = from.physicalOptimizationConfig;
+        defaultNodeDomain = from.defaultNodeDomain;
+        prettyPrinter = from.prettyPrinter;
+        conflictingTypeResovler = from.conflictingTypeResovler;
+        warningCollector = NoOpWarningCollector.INSTANCE;
+        boolean isSanityCheckEnabled = physicalOptimizationConfig.isSanityCheckEnabled();
+        planStructureVerifier = isSanityCheckEnabled ? new PlanStructureVerifier(from.prettyPrinter, this) : null;
+        planStabilityVerifier = isSanityCheckEnabled ? new PlanStabilityVerifier(from.prettyPrinter) : null;
+        metadataProvider = from.metadataProvider;
+        compilerFactory = from.compilerFactory;
+
+        varEvalSizeEnv.varSizeMap.putAll(from.varEvalSizeEnv.varSizeMap);
+        typeEnvMap.putAll(from.typeEnvMap);
+        alreadyCompared.putAll(from.alreadyCompared);
+        dontApply.putAll(from.dontApply);
+        varToPrimaryKey.putAll(from.varToPrimaryKey);
+        notToBeInlinedVars.addAll(from.notToBeInlinedVars);
+        fdGlobalMap.putAll(from.fdGlobalMap);
+        eqClassGlobalMap.putAll(from.eqClassGlobalMap);
+        logicalProps.putAll(from.logicalProps);
+    }
+
     @Override
     public int getVarCounter() {
         return varCounter;
@@ -354,4 +373,34 @@
     public PlanStabilityVerifier getPlanStabilityVerifier() {
         return planStabilityVerifier;
     }
+
+    @Override
+    public IOptimizationContextFactory getOptimizationContextFactory() {
+        return optContextFactory;
+    }
+
+    @Override
+    public void setCompilerFactory(Object compilerFactory) {
+        this.compilerFactory = compilerFactory;
+    }
+
+    @Override
+    public Object getCompilerFactory() {
+        return compilerFactory;
+    }
+
+    protected static class VariableEvalSizeEnvironmentImpl implements IVariableEvalSizeEnvironment {
+
+        protected final Map<LogicalVariable, Integer> varSizeMap = new HashMap<>();
+
+        @Override
+        public void setVariableEvalSize(LogicalVariable var, int size) {
+            varSizeMap.put(var, size);
+        }
+
+        @Override
+        public int getVariableEvalSize(LogicalVariable var) {
+            return varSizeMap.get(var);
+        }
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IOptimizationContextFactory.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IOptimizationContextFactory.java
index 1c41e9a..c81025d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IOptimizationContextFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IOptimizationContextFactory.java
@@ -28,10 +28,12 @@
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 
 public interface IOptimizationContextFactory {
-    public IOptimizationContext createOptimizationContext(int varCounter,
+    IOptimizationContext createOptimizationContext(int varCounter,
             IExpressionEvalSizeComputer expressionEvalSizeComputer,
             IMergeAggregationExpressionFactory mergeAggregationExpressionFactory,
             IExpressionTypeComputer expressionTypeComputer, IMissableTypeComputer missableTypeComputer,
             IConflictingTypeResolver conflictintTypeResolver, PhysicalOptimizationConfig physicalOptimizationConfig,
             AlgebricksPartitionConstraint clusterLocations, IWarningCollector warningCollector);
+
+    IOptimizationContext cloneOptimizationContext(IOptimizationContext oc);
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IRuleSetKind.java
similarity index 83%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IRuleSetKind.java
index 6095b26..5ce61ec 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/IRuleSetKind.java
@@ -17,8 +17,8 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+package org.apache.hyracks.algebricks.core.rewriter.base;
 
-select value v from range(1,2) v where v > ?;
+public interface IRuleSetKind {
+    String name();
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
index 9c9b21d..4d54a10 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 final Properties properties = new Properties();
 
@@ -239,6 +240,14 @@
         return getInt(EXTERNAL_SCAN_BUFFER_SIZE, AlgebricksConfig.EXTERNAL_SCAN_BUFFER_SIZE);
     }
 
+    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-data/src/main/java/org/apache/hyracks/algebricks/data/IResultSerializerFactoryProvider.java b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IResultSerializerFactoryProvider.java
index 7525180..75fd791 100644
--- a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IResultSerializerFactoryProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IResultSerializerFactoryProvider.java
@@ -32,10 +32,8 @@
      *            - A printer factory array to print the tuple containing different fields.
      * @param writerFactory
      *            - A writer factory to write the serialized data to the print stream.
-     * @param inputRecordDesc
-     *            - The record descriptor describing the input frame to be serialized.
      * @return A new instance of result serialized appender.
      */
-    public IResultSerializerFactory getAqlResultSerializerFactoryProvider(int[] fields,
-            IPrinterFactory[] printerFactories, IAWriterFactory writerFactory);
+    public IResultSerializerFactory getResultSerializerFactoryProvider(int[] fields, IPrinterFactory[] printerFactories,
+            IAWriterFactory writerFactory);
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
index e13ec30..129aba3 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IsolateHyracksOperatorsRule.java
@@ -132,6 +132,7 @@
         e.setExecutionMode(inOp.getExecutionMode());
         context.computeAndSetTypeEnvironmentForOperator(e);
         e.recomputeSchema();
+        e.computeDeliveredPhysicalProperties(context);
 
         inOpRef.setValue(e);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java
index e1b2e5d..1eda133 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java
@@ -47,7 +47,7 @@
         }
         DistributeResultOperator dop = (DistributeResultOperator) op;
         IResultMetadata resultMetadata = dop.getResultMetadata();
-        if (resultMetadata.getOutputTypes() != null) {
+        if (resultMetadata == null || resultMetadata.getOutputTypes() != null) {
             return false;
         }
         List<Mutable<ILogicalExpression>> exprList = dop.getExpressions();
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java
index 3760be5..1ed7170 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java
@@ -138,9 +138,12 @@
                 if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
                     continue;
                 }
-                VariableReferenceExpression rhsVarRefExpr = (VariableReferenceExpression) expr;
-                // Update equivalence class map.
                 LogicalVariable lhs = assignOp.getVariables().get(i);
+                if (context.shouldNotBeInlined(lhs)) {
+                    continue;
+                }
+                // Update equivalence class map.
+                VariableReferenceExpression rhsVarRefExpr = (VariableReferenceExpression) expr;
                 LogicalVariable rhs = rhsVarRefExpr.getVariableReference();
                 updateEquivalenceClassMap(lhs, rhs);
             }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java
index 763e6ff..90fa824 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java
@@ -40,7 +40,7 @@
     }
 
     @Override
-    public IResultSerializerFactory getAqlResultSerializerFactoryProvider(final int[] fields,
+    public IResultSerializerFactory getResultSerializerFactoryProvider(final int[] fields,
             final IPrinterFactory[] printerFactories, final IAWriterFactory writerFactory) {
         return new IResultSerializerFactory() {
             private static final long serialVersionUID = 1L;
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/writers/SerializedDataWriterFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/writers/SerializedDataWriterFactory.java
index bc7634d..8ce4f2b 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/writers/SerializedDataWriterFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/writers/SerializedDataWriterFactory.java
@@ -32,7 +32,15 @@
 
 public class SerializedDataWriterFactory implements IAWriterFactory {
 
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 2L;
+
+    public static final SerializedDataWriterFactory WITHOUT_RECORD_DESCRIPTOR = new SerializedDataWriterFactory(false);
+
+    private final boolean writeRecordDescriptor;
+
+    public SerializedDataWriterFactory(boolean writeRecordDescriptor) {
+        this.writeRecordDescriptor = writeRecordDescriptor;
+    }
 
     @Override
     public IAWriter createWriter(final int[] fields, final PrintStream ps, IPrinterFactory[] printerFactories,
@@ -41,15 +49,17 @@
 
             @Override
             public void init() throws HyracksDataException {
-                // dump the SerializerDeserializers to disk
-                try {
-                    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-                    ObjectOutputStream oos = new ObjectOutputStream(baos);
-                    oos.writeObject(inputRecordDescriptor);
-                    baos.writeTo(ps);
-                    oos.close();
-                } catch (IOException e) {
-                    throw HyracksDataException.create(e);
+                if (writeRecordDescriptor) {
+                    // dump the SerializerDeserializers to disk
+                    try {
+                        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+                        ObjectOutputStream oos = new ObjectOutputStream(baos);
+                        oos.writeObject(inputRecordDescriptor);
+                        baos.writeTo(ps);
+                        oos.close();
+                    } catch (IOException e) {
+                        throw HyracksDataException.create(e);
+                    }
                 }
             }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
index 0ee3658..7a320b1 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
@@ -346,12 +346,19 @@
     }
 
     public static class WaitForCompletionFunction extends Function {
-        private static final long serialVersionUID = 1L;
+        private static final long serialVersionUID = 2L;
 
         private final JobId jobId;
 
+        private final List<String> statOperatorNames;
+
         public WaitForCompletionFunction(JobId jobId) {
+            this(jobId, null);
+        }
+
+        public WaitForCompletionFunction(JobId jobId, List<String> statOperatorNames) {
             this.jobId = jobId;
+            this.statOperatorNames = statOperatorNames;
         }
 
         @Override
@@ -362,6 +369,10 @@
         public JobId getJobId() {
             return jobId;
         }
+
+        public List<String> getStatOperatorNames() {
+            return statOperatorNames;
+        }
     }
 
     public static class GetNodeControllersInfoFunction extends Function {
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
index 1118a68..be965eb 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
@@ -31,6 +31,7 @@
 import org.apache.hyracks.api.job.JobInfo;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 
 /**
  * Interface used by clients to communicate with the Hyracks Cluster Controller.
@@ -157,6 +158,19 @@
     void waitForCompletion(JobId jobId) throws Exception;
 
     /**
+     * Waits until the specified job has completed, either successfully or has
+     * encountered a permanent failure.
+     *
+     * @param jobId
+     *            JobId of the Job
+     * @param statsOperatorNames
+     *            names of the operators which stats should be returned
+     * @return stats for the specified operators
+     * @throws Exception
+     */
+    List<IOperatorStats> waitForCompletion(JobId jobId, List<String> statsOperatorNames) throws Exception;
+
+    /**
      * Deploy files to the cluster
      *
      * @param files
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
index 4cc47d2..be47014 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
@@ -30,6 +30,7 @@
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobInfo;
 import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 import org.apache.hyracks.api.topology.ClusterTopology;
 
 public interface IHyracksClientInterface {
@@ -53,6 +54,8 @@
 
     public void waitForCompletion(JobId jobId) throws Exception;
 
+    public List<IOperatorStats> waitForCompletion(JobId jobId, List<String> statOperatorNames) throws Exception;
+
     public Map<String, NodeControllerInfo> getNodeControllersInfo() throws Exception;
 
     public ClusterTopology getClusterTopology() throws Exception;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
index 4561af1..ad5624c 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
@@ -42,6 +42,8 @@
 
     TaskAttemptId getTaskAttemptId();
 
+    int getPartitionCount();
+
     ICounterContext getCounterContext();
 
     ExecutorService getExecutorService();
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/NoOpWarningCollector.java
similarity index 62%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/NoOpWarningCollector.java
index 6095b26..caf2464 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/NoOpWarningCollector.java
@@ -17,8 +17,27 @@
  * under the License.
  */
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+package org.apache.hyracks.api.exceptions;
 
-select value v from range(1,2) v where v > ?;
+public final class NoOpWarningCollector implements IWarningCollector {
+
+    public static final IWarningCollector INSTANCE = new NoOpWarningCollector();
+
+    private NoOpWarningCollector() {
+    }
+
+    @Override
+    public void warn(Warning warning) {
+        // no-op
+    }
+
+    @Override
+    public boolean shouldWarn() {
+        return false;
+    }
+
+    @Override
+    public long getTotalWarningsCount() {
+        return 0;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IStatsCollector.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/profiling/IStatsCollector.java
index 8930d34..c903c68 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
@@ -43,6 +43,13 @@
     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
+     */
+    IOperatorStats getOperatorStats(String operatorName);
+
+    /**
      * Get every registered operator stats object
      * @return All registered operators, and their collected stats, with the names as keys and stats as values
      */
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksConstants.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksConstants.java
index c34a671..3c829e6 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksConstants.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksConstants.java
@@ -28,6 +28,8 @@
 
     public static final String INDEX_CURSOR_STATS = "INDEX_CURSOR_STATS";
 
+    public static final String TUPLE_PROJECTOR = "TUPLE_PROJECTOR";
+
     private HyracksConstants() {
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-client/pom.xml b/hyracks-fullstack/hyracks/hyracks-client/pom.xml
index 409da1f8..66b71fa 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-client/pom.xml
@@ -101,6 +101,11 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-util</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java
index b335a93..5dd5fff 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java
+++ b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultDirectory.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hyracks.client.result;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 
@@ -35,7 +36,7 @@
 import org.apache.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
 
 //TODO(madhusudancs): Should this implementation be moved to org.apache.hyracks.client?
-public class ResultDirectory implements IResultDirectory {
+public class ResultDirectory implements IResultDirectory, Closeable {
     private final IPCSystem ipc;
     private final IResultDirectory remoteResultDirectory;
 
@@ -64,4 +65,9 @@
     public IResultMetadata getResultMetadata(JobId jobId, ResultSetId rsId) throws Exception {
         return remoteResultDirectory.getResultMetadata(jobId, rsId);
     }
+
+    @Override
+    public void close() throws IOException {
+        ipc.stop();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java
index 4d8767f..8a88045 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java
+++ b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSet.java
@@ -18,6 +18,9 @@
  */
 package org.apache.hyracks.client.result;
 
+import java.io.Closeable;
+import java.io.IOException;
+
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.comm.NetworkAddress;
 import org.apache.hyracks.api.context.IHyracksCommonContext;
@@ -25,15 +28,15 @@
 import org.apache.hyracks.api.io.IIOManager;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.network.ISocketChannelFactory;
-import org.apache.hyracks.api.result.IResultDirectory;
 import org.apache.hyracks.api.result.IResultSet;
 import org.apache.hyracks.api.result.IResultSetReader;
 import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.client.net.ClientNetworkManager;
 import org.apache.hyracks.control.nc.resources.memory.FrameManager;
+import org.apache.hyracks.util.NetworkUtil;
 
-public class ResultSet implements IResultSet {
-    private final IResultDirectory resultDirectory;
+public class ResultSet implements IResultSet, Closeable {
+    private final ResultDirectory resultDirectory;
 
     private final ClientNetworkManager netManager;
 
@@ -51,6 +54,15 @@
     }
 
     @Override
+    public void close() throws IOException {
+        try {
+            netManager.stop();
+        } finally {
+            NetworkUtil.closeQuietly(resultDirectory);
+        }
+    }
+
+    @Override
     public IResultSetReader createReader(JobId jobId, ResultSetId resultSetId) throws HyracksDataException {
         IResultSetReader reader = null;
         try {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
index e6973dd..6da8fdf 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
@@ -147,8 +147,8 @@
             case WAIT_FOR_COMPLETION:
                 HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf =
                         (HyracksClientInterfaceFunctions.WaitForCompletionFunction) fn;
-                ccs.getWorkQueue()
-                        .schedule(new WaitForJobCompletionWork(ccs, wfcf.getJobId(), new IPCResponder<>(handle, mid)));
+                ccs.getWorkQueue().schedule(new WaitForJobCompletionWork(ccs, wfcf.getJobId(),
+                        wfcf.getStatOperatorNames(), new IPCResponder<>(handle, mid)));
                 break;
             case GET_NODE_CONTROLLERS_INFO:
                 ccs.getWorkQueue().schedule(
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
index 911bedf..14d92fb 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
@@ -33,11 +33,14 @@
 public class WaitForJobCompletionWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final JobId jobId;
+    private final List<String> statOperatorNames;
     private final IResultCallback<Object> callback;
 
-    public WaitForJobCompletionWork(ClusterControllerService ccs, JobId jobId, IResultCallback<Object> callback) {
+    public WaitForJobCompletionWork(ClusterControllerService ccs, JobId jobId, List<String> statOperatorNames,
+            IResultCallback<Object> callback) {
         this.ccs = ccs;
         this.jobId = jobId;
+        this.statOperatorNames = statOperatorNames;
         this.callback = callback;
     }
 
@@ -53,7 +56,7 @@
                         Thread.currentThread()
                                 .setName(Thread.currentThread().getName() + " : WaitForCompletionForJobId: " + jobId);
                         jobRun.waitForCompletion();
-                        callback.setValue(null);
+                        callback.setValue(jobRun.getJobProfile().getAggregatedStats(statOperatorNames));
                     } catch (Exception e) {
                         callback.setException(e);
                     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/StatsCollector.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/StatsCollector.java
index 878d7c3..580a560 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
@@ -52,6 +52,11 @@
     }
 
     @Override
+    public IOperatorStats getOperatorStats(String operatorName) {
+        return operatorStatsMap.get(operatorName);
+    }
+
+    @Override
     public Map<String, IOperatorStats> getAllOperatorStats() {
         return Collections.unmodifiableMap(operatorStatsMap);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/JobProfile.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/JobProfile.java
index c4eff85..0026af4 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/JobProfile.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/job/profiling/om/JobProfile.java
@@ -21,11 +21,18 @@
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.hyracks.api.dataflow.TaskAttemptId;
+import org.apache.hyracks.api.dataflow.TaskId;
 import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.job.profiling.IStatsCollector;
+import org.apache.hyracks.api.job.profiling.OperatorStats;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ArrayNode;
@@ -109,4 +116,44 @@
             entry.getValue().writeFields(output);
         }
     }
+
+    public List<IOperatorStats> getAggregatedStats(List<String> operatorNames) {
+        if (jobletProfiles == null || operatorNames == null || operatorNames.isEmpty()) {
+            return null;
+        }
+        // gather final task attempts for each task
+        Map<TaskId, TaskProfile> taskProfileMap = new HashMap<>();
+        for (JobletProfile jobletProfile : jobletProfiles.values()) {
+            for (TaskProfile taskProfile : jobletProfile.getTaskProfiles().values()) {
+                TaskAttemptId taskAttemptId = taskProfile.getTaskId();
+                TaskId taskId = taskAttemptId.getTaskId();
+                TaskProfile existingProfile = taskProfileMap.get(taskId);
+                if (existingProfile == null || taskAttemptId.getAttempt() > existingProfile.getTaskId().getAttempt()) {
+                    taskProfileMap.put(taskId, taskProfile);
+                }
+            }
+        }
+        // compute aggregated counts
+        int n = operatorNames.size();
+        IOperatorStats[] outStats = new IOperatorStats[n];
+        for (TaskProfile taskProfile : taskProfileMap.values()) {
+            IStatsCollector statsCollector = taskProfile.getStatsCollector();
+            for (int i = 0; i < n; i++) {
+                String operatorName = operatorNames.get(i);
+                IOperatorStats opTaskStats = statsCollector.getOperatorStats(operatorName);
+                if (opTaskStats == null) {
+                    continue;
+                }
+                IOperatorStats opOutStats = outStats[i];
+                if (opOutStats == null) {
+                    opOutStats = new OperatorStats(operatorName);
+                    outStats[i] = opOutStats;
+                }
+                opOutStats.getTupleCounter().update(opTaskStats.getTupleCounter().get());
+                opOutStats.getTimeCounter().update(opTaskStats.getTimeCounter().get());
+                opOutStats.getDiskIoCounter().update(opTaskStats.getDiskIoCounter().get());
+            }
+        }
+        return Arrays.asList(outStats);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
index 158e24e..900ac7e 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
@@ -87,6 +87,8 @@
 
     private final TaskAttemptId taskAttemptId;
 
+    private final int partitionCount;
+
     private final String displayName;
 
     private final ExecutorService executorService;
@@ -131,12 +133,13 @@
 
     private final Map<Long, IThreadStats> perThreadStats = new HashMap<>();
 
-    public Task(Joblet joblet, Set<JobFlag> jobFlags, TaskAttemptId taskId, String displayName,
+    public Task(Joblet joblet, Set<JobFlag> jobFlags, TaskAttemptId taskId, int partitionCount, String displayName,
             ExecutorService executor, NodeControllerService ncs,
             List<List<PartitionChannel>> inputChannelsFromConnectors) {
         this.joblet = joblet;
         this.jobFlags = jobFlags;
         this.taskAttemptId = taskId;
+        this.partitionCount = partitionCount;
         this.displayName = displayName;
         this.executorService = executor;
         fileFactory = new WorkspaceFileFactory(this, joblet.getIoManager());
@@ -225,6 +228,11 @@
     }
 
     @Override
+    public int getPartitionCount() {
+        return partitionCount;
+    }
+
+    @Override
     public ICounter getCounter(String name, boolean create) {
         Counter counter = counterMap.get(name);
         if (counter == null && create) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
index 7d84fe9..f6c144d 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
@@ -152,11 +152,12 @@
                 ActivityCluster ac = acg.getActivityMap().get(aid);
                 IActivity han = ac.getActivityMap().get(aid);
                 LOGGER.trace("Initializing {} -> {} for {}", taId, han, jobId);
+                int partitionCount = td.getPartitionCount();
                 final int partition = tid.getPartition();
                 List<IConnectorDescriptor> inputs = ac.getActivityInputMap().get(aid);
-                task = new Task(joblet, flags, taId, han.getDisplayName(), ncs.getExecutor(), ncs,
+                task = new Task(joblet, flags, taId, partitionCount, han.getDisplayName(), ncs.getExecutor(), ncs,
                         createInputChannels(td, inputs));
-                IOperatorNodePushable operator = han.createPushRuntime(task, rdp, partition, td.getPartitionCount());
+                IOperatorNodePushable operator = han.createPushRuntime(task, rdp, partition, partitionCount);
                 List<IPartitionCollector> collectors = new ArrayList<>();
                 if (inputs != null) {
                     for (int i = 0; i < inputs.size(); ++i) {
@@ -182,7 +183,7 @@
                                 createPartitionWriterFactory(task, cPolicy, jobId, conn, partition, taId, flags);
                         LOGGER.trace("input: {}: {}", i, conn.getConnectorId());
                         IFrameWriter writer = conn.createPartitioner(task, recordDesc, pwFactory, partition,
-                                td.getPartitionCount(), td.getOutputPartitionCounts()[i]);
+                                partitionCount, td.getOutputPartitionCounts()[i]);
                         writer = (enforce && !profile) ? EnforceFrameWriter.enforce(writer) : writer;
                         operator.setOutputFrameWriter(i, writer, recordDesc);
                     }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
index c370b58..26b9f10 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
@@ -69,7 +69,7 @@
 
     @Override
     public void close() throws HyracksDataException {
-        if (!failed) {
+        if (!failed && handle != null) {
             ioManager.close(handle);
         }
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
index bcd71a3..c3bc5ca 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
@@ -49,6 +49,6 @@
                 getVirtualBufferCacheProvider(), SynchronousSchedulerProvider.INSTANCE, MERGE_POLICY_FACTORY,
                 MERGE_POLICY_PROPERTIES, DURABLE, bloomFilterKeyFields,
                 LSMTreeOperatorTestHelper.DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, true, btreefields,
-                NoOpCompressorDecompressorFactory.INSTANCE, bloomFilterKeyFields != null, null, null);
+                NoOpCompressorDecompressorFactory.INSTANCE, bloomFilterKeyFields != null, null, null, false);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java
index 4732d71..1f5b441 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/CLFLogger.java
@@ -43,7 +43,7 @@
 //Based in part on LoggingHandler from Netty
 public class CLFLogger extends ChannelDuplexHandler {
 
-    private static final Logger accessLogger = LogManager.getLogger();
+    private static final Logger LOGGER = LogManager.getLogger();
     private static final Level ACCESS_LOG_LEVEL = Level.forName("ACCESS", 550);
     private static final DateTimeFormatter DATE_TIME_FORMATTER =
             DateTimeFormatter.ofPattern("dd/MMM/yyyy:HH:mm:ss Z").withZone(ZoneId.systemDefault());
@@ -66,9 +66,14 @@
     public void channelRead(ChannelHandlerContext ctx, Object msg) {
         if (msg instanceof HttpRequest) {
             HttpRequest req = (HttpRequest) msg;
-            clientIp = ((NioSocketChannel) ctx.channel()).remoteAddress().getAddress().toString().substring(1);
+            try {
+                clientIp = ((NioSocketChannel) ctx.channel()).remoteAddress().getAddress().toString().substring(1);
+            } catch (Exception e) {
+                LOGGER.debug("ignoring {} obtaining client ip for {}", e, ctx.channel());
+                clientIp = "-";
+            }
             requestTime = Instant.now();
-            reqLine = req.method().toString() + " " + req.getUri() + " " + req.getProtocolVersion().toString();
+            reqLine = req.method().toString() + " " + req.uri() + " " + req.protocolVersion();
             userAgentRef = headerValueOrDash("Referer", req) + " " + headerValueOrDash("User-Agent", req);
             lastChunk = false;
         }
@@ -116,7 +121,7 @@
     }
 
     private void printAndPrepare() {
-        if (!accessLogger.isEnabled(ACCESS_LOG_LEVEL)) {
+        if (!LOGGER.isEnabled(ACCESS_LOG_LEVEL)) {
             return;
         }
         logLineBuilder.append(clientIp);
@@ -131,7 +136,7 @@
         logLineBuilder.append(" ").append(statusCode);
         logLineBuilder.append(" ").append(respSize);
         logLineBuilder.append(" ").append(userAgentRef);
-        accessLogger.log(ACCESS_LOG_LEVEL, logLineBuilder);
+        LOGGER.log(ACCESS_LOG_LEVEL, logLineBuilder);
         respSize = 0;
         logLineBuilder.setLength(0);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksClientInterfaceRemoteProxy.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksClientInterfaceRemoteProxy.java
index 82d7a3b..3b59a70 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksClientInterfaceRemoteProxy.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksClientInterfaceRemoteProxy.java
@@ -35,6 +35,7 @@
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobInfo;
 import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 import org.apache.hyracks.api.topology.ClusterTopology;
 import org.apache.hyracks.ipc.api.IIPCHandle;
 import org.apache.hyracks.ipc.api.RPCInterface;
@@ -130,6 +131,13 @@
     }
 
     @Override
+    public List<IOperatorStats> waitForCompletion(JobId jobId, List<String> statOperatorNames) throws Exception {
+        HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf =
+                new HyracksClientInterfaceFunctions.WaitForCompletionFunction(jobId, statOperatorNames);
+        return (List<IOperatorStats>) rpci.call(ipcHandle, wfcf);
+    }
+
+    @Override
     public Map<String, NodeControllerInfo> getNodeControllersInfo() throws Exception {
         HyracksClientInterfaceFunctions.GetNodeControllersInfoFunction gncif =
                 new HyracksClientInterfaceFunctions.GetNodeControllersInfoFunction();
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksConnection.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksConnection.java
index 9351348..1b6d8af 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HyracksConnection.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hyracks.ipc.impl;
 
+import java.io.Closeable;
 import java.io.File;
 import java.net.InetSocketAddress;
 import java.net.URL;
@@ -52,6 +53,7 @@
 import org.apache.hyracks.api.job.JobInfo;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
 import org.apache.hyracks.api.network.ISocketChannelFactory;
 import org.apache.hyracks.api.topology.ClusterTopology;
 import org.apache.hyracks.api.util.InvokeUtil;
@@ -70,7 +72,7 @@
  *
  * @author vinayakb
  */
-public final class HyracksConnection implements IHyracksClientConnection {
+public final class HyracksConnection implements Closeable, IHyracksClientConnection {
 
     private static final Logger LOGGER = LogManager.getLogger();
 
@@ -123,6 +125,11 @@
     }
 
     @Override
+    public void close() {
+        ipc.stop();
+    }
+
+    @Override
     public JobStatus getJobStatus(JobId jobId) throws Exception {
         return hci.getJobStatus(jobId);
     }
@@ -196,6 +203,17 @@
     }
 
     @Override
+    public List<IOperatorStats> waitForCompletion(JobId jobId, List<String> statOperatorNames) throws Exception {
+        try {
+            return hci.waitForCompletion(jobId, statOperatorNames);
+        } catch (InterruptedException e) {
+            // Cancels an on-going job if the current thread gets interrupted.
+            cancelJob(jobId);
+            throw e;
+        }
+    }
+
+    @Override
     public Map<String, NodeControllerInfo> getNodeControllerInfos() throws HyracksException {
         try {
             return hci.getNodeControllersInfo();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.java
new file mode 100644
index 0000000..f436636
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/api/IDiskBTreeStatefulPointSearchCursor.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.btree.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.DiskBTreePointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+
+/**
+ * Allows stateful {@link DiskBTreePointSearchCursor} to resume the search without closing and reopening the cursor
+ * Note: upon closing a cursor, ensure the search state is cleared
+ */
+public interface IDiskBTreeStatefulPointSearchCursor {
+    int getLastPageId();
+
+    void setCursorToNextKey(ISearchPredicate searchPred) throws HyracksDataException;
+
+    ITreeIndexFrame getFrame();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 2455625..0ab88a5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -29,10 +29,12 @@
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
 
 public class BTreeSearchOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
 
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 2L;
 
     protected final int[] lowKeyFields;
     protected final int[] highKeyFields;
@@ -52,6 +54,7 @@
     protected byte[] searchCallbackProceedResultTrueValue;
     protected final ITupleFilterFactory tupleFilterFactory;
     protected final long outputLimit;
+    protected final ITupleProjectorFactory tupleProjectorFactory;
 
     public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
             int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
@@ -61,7 +64,8 @@
             IMissingWriterFactory nonFilterWriterFactory) {
         this(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
-                maxFilterFieldIndexes, appendIndexFilter, nonFilterWriterFactory, null, -1, false, null, null);
+                maxFilterFieldIndexes, appendIndexFilter, nonFilterWriterFactory, null, -1, false, null, null,
+                DefaultTupleProjectorFactory.INSTANCE);
     }
 
     public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
@@ -71,7 +75,7 @@
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter,
             IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFilterFactory, long outputLimit,
             boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
-            byte[] searchCallbackProceedResultTrueValue) {
+            byte[] searchCallbackProceedResultTrueValue, ITupleProjectorFactory tupleProjectorFactory) {
         super(spec, 1, 1);
         this.indexHelperFactory = indexHelperFactory;
         this.retainInput = retainInput;
@@ -92,6 +96,7 @@
         this.appendOpCallbackProceedResult = appendOpCallbackProceedResult;
         this.searchCallbackProceedResultFalseValue = searchCallbackProceedResultFalseValue;
         this.searchCallbackProceedResultTrueValue = searchCallbackProceedResultTrueValue;
+        this.tupleProjectorFactory = tupleProjectorFactory;
     }
 
     @Override
@@ -102,7 +107,7 @@
                 lowKeyInclusive, highKeyInclusive, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
                 nonFilterWriterFactory, tupleFilterFactory, outputLimit, appendOpCallbackProceedResult,
-                searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
+                searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue, tupleProjectorFactory);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index fb331bf..a56e305 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -22,6 +22,7 @@
 import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.HyracksConstants;
 import org.apache.hyracks.dataflow.common.data.accessors.PermutingFrameTupleReference;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
@@ -30,9 +31,11 @@
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IndexSearchOperatorNodePushable;
+import org.apache.hyracks.storage.am.common.impls.DefaultTupleProjectorFactory;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
 
 public class BTreeSearchOperatorNodePushable extends IndexSearchOperatorNodePushable {
     protected final boolean lowKeyInclusive;
@@ -52,7 +55,7 @@
         this(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
                 minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing,
                 nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter, nonFilterWriterFactory, null, -1,
-                false, null, null);
+                false, null, null, DefaultTupleProjectorFactory.INSTANCE);
     }
 
     public BTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
@@ -62,11 +65,12 @@
             ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
             IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFilterFactory, long outputLimit,
             boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
-            byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
+            byte[] searchCallbackProceedResultTrueValue, ITupleProjectorFactory projectorFactory)
+            throws HyracksDataException {
         super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
                 retainInput, retainMissing, nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter,
                 nonFilterWriterFactory, tupleFilterFactory, outputLimit, appendOpCallbackProceedResult,
-                searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
+                searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue, projectorFactory);
         this.lowKeyInclusive = lowKeyInclusive;
         this.highKeyInclusive = highKeyInclusive;
         if (lowKeyFields != null && lowKeyFields.length > 0) {
@@ -111,7 +115,8 @@
 
     @Override
     protected void addAdditionalIndexAccessorParams(IIndexAccessParameters iap) throws HyracksDataException {
-        // No additional parameters are required for the B+Tree search case
+        //Set tuple projector to get the information about the pushed down value accesses (if supported by the index)
+        iap.getParameters().put(HyracksConstants.TUPLE_PROJECTOR, tupleProjector);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
index 78faaff..40b2f5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
@@ -42,7 +42,6 @@
 import org.apache.hyracks.storage.am.btree.impls.BTreeOpContext.PageValidationInfo;
 import org.apache.hyracks.storage.am.common.api.IBTreeIndexTupleReference;
 import org.apache.hyracks.storage.am.common.api.IPageManager;
-import org.apache.hyracks.storage.am.common.api.ISplitKey;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
@@ -51,7 +50,6 @@
 import org.apache.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
 import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
-import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
 import org.apache.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
@@ -64,17 +62,13 @@
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
 import org.apache.hyracks.storage.common.NoOpIndexCursorStats;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
-import org.apache.hyracks.util.JSONUtil;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
 public class BTree extends AbstractTreeIndex {
 
     public static final float DEFAULT_FILL_FACTOR = 0.7f;
@@ -85,7 +79,7 @@
 
     private final AtomicInteger smoCounter;
     private final ReadWriteLock treeLatch;
-    private final int maxTupleSize;
+    protected final int maxTupleSize;
 
     public BTree(IBufferCache bufferCache, IPageManager freePageManager, ITreeIndexFrameFactory interiorFrameFactory,
             ITreeIndexFrameFactory leafFrameFactory, IBinaryComparatorFactory[] cmpFactories, int fieldCount,
@@ -886,13 +880,13 @@
         }
 
         @Override
-        public BTreeRangeSearchCursor createSearchCursor(boolean exclusive) {
+        public ITreeIndexCursor createSearchCursor(boolean exclusive) {
             IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
             return new BTreeRangeSearchCursor(leafFrame, exclusive, (IIndexCursorStats) iap.getParameters()
                     .getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE));
         }
 
-        public BTreeRangeSearchCursor createPointCursor(boolean exclusive, boolean stateful) {
+        public ITreeIndexCursor createPointCursor(boolean exclusive, boolean stateful) {
             return createSearchCursor(exclusive);
         }
 
@@ -1004,220 +998,7 @@
     @Override
     public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
             boolean checkIfEmptyIndex, IPageWriteCallback callback) throws HyracksDataException {
-        return new BTreeBulkLoader(fillFactor, verifyInput, callback);
-    }
-
-    public class BTreeBulkLoader extends AbstractTreeIndex.AbstractTreeIndexBulkLoader {
-        protected final ISplitKey splitKey;
-        protected final boolean verifyInput;
-
-        public BTreeBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback)
-                throws HyracksDataException {
-            super(fillFactor, callback);
-            this.verifyInput = verifyInput;
-            splitKey = new BTreeSplitKey(leafFrame.getTupleWriter().createTupleReference());
-            splitKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
-        }
-
-        @Override
-        public void add(ITupleReference tuple) throws HyracksDataException {
-            try {
-                int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
-                        interiorFrame.getBytesRequiredToWriteTuple(tuple));
-                NodeFrontier leafFrontier = nodeFrontiers.get(0);
-                int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
-                int spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
-
-                // try to free space by compression
-                if (spaceUsed + spaceNeeded > leafMaxBytes) {
-                    leafFrame.compress();
-                    spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
-                }
-                //full, allocate new page
-                if (spaceUsed + spaceNeeded > leafMaxBytes) {
-                    if (leafFrame.getTupleCount() == 0) {
-                        bufferCache.returnPage(leafFrontier.page, false);
-                    } else {
-                        leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
-                        if (verifyInput) {
-                            verifyInputTuple(tuple, leafFrontier.lastTuple);
-                        }
-                        int splitKeySize = tupleWriter.bytesRequired(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount());
-                        splitKey.initData(splitKeySize);
-                        tupleWriter.writeTupleFields(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount(),
-                                splitKey.getBuffer().array(), 0);
-                        splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
-                        splitKey.setLeftPage(leafFrontier.pageId);
-
-                        propagateBulk(1, pagesToWrite);
-
-                        leafFrontier.pageId = freePageManager.takePage(metaFrame);
-
-                        ((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId);
-
-                        write(leafFrontier.page);
-                        for (ICachedPage c : pagesToWrite) {
-                            write(c);
-                        }
-                        pagesToWrite.clear();
-                        splitKey.setRightPage(leafFrontier.pageId);
-                    }
-                    if (tupleSize > maxTupleSize) {
-                        final long dpid = BufferedFileHandle.getDiskPageId(getFileId(), leafFrontier.pageId);
-                        // calculate required number of pages.
-                        int headerSize = Math.max(leafFrame.getPageHeaderSize(), interiorFrame.getPageHeaderSize());
-                        final int multiplier =
-                                (int) Math.ceil((double) tupleSize / (bufferCache.getPageSize() - headerSize));
-                        if (multiplier > 1) {
-                            leafFrontier.page = bufferCache.confiscateLargePage(dpid, multiplier,
-                                    freePageManager.takeBlock(metaFrame, multiplier - 1));
-                        } else {
-                            leafFrontier.page = bufferCache.confiscatePage(dpid);
-                        }
-                        leafFrame.setPage(leafFrontier.page);
-                        leafFrame.initBuffer((byte) 0);
-                        ((IBTreeLeafFrame) leafFrame).setLargeFlag(true);
-                    } else {
-                        final long dpid = BufferedFileHandle.getDiskPageId(getFileId(), leafFrontier.pageId);
-                        leafFrontier.page = bufferCache.confiscatePage(dpid);
-                        leafFrame.setPage(leafFrontier.page);
-                        leafFrame.initBuffer((byte) 0);
-                    }
-                } else {
-                    if (verifyInput && leafFrame.getTupleCount() > 0) {
-                        leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
-                        verifyInputTuple(tuple, leafFrontier.lastTuple);
-                    }
-                }
-                ((IBTreeLeafFrame) leafFrame).insertSorted(tuple);
-            } catch (HyracksDataException | RuntimeException e) {
-                logState(tuple, e);
-                handleException();
-                throw e;
-            }
-        }
-
-        protected void verifyInputTuple(ITupleReference tuple, ITupleReference prevTuple) throws HyracksDataException {
-            // New tuple should be strictly greater than last tuple.
-            int cmpResult = cmp.compare(tuple, prevTuple);
-            if (cmpResult < 0) {
-                throw HyracksDataException.create(ErrorCode.UNSORTED_LOAD_INPUT);
-            }
-            if (cmpResult == 0) {
-                throw HyracksDataException.create(ErrorCode.DUPLICATE_LOAD_INPUT);
-            }
-        }
-
-        protected void propagateBulk(int level, List<ICachedPage> pagesToWrite) throws HyracksDataException {
-            if (splitKey.getBuffer() == null) {
-                return;
-            }
-
-            if (level >= nodeFrontiers.size()) {
-                addLevel();
-            }
-
-            NodeFrontier frontier = nodeFrontiers.get(level);
-            interiorFrame.setPage(frontier.page);
-
-            ITupleReference tuple = splitKey.getTuple();
-            int tupleBytes = tupleWriter.bytesRequired(tuple, 0, cmp.getKeyFieldCount());
-            int spaceNeeded = tupleBytes + slotSize + 4;
-            if (tupleBytes > interiorFrame.getMaxTupleSize(BTree.this.bufferCache.getPageSize())) {
-                throw HyracksDataException.create(ErrorCode.RECORD_IS_TOO_LARGE, tupleBytes,
-                        interiorFrame.getMaxTupleSize(BTree.this.bufferCache.getPageSize()));
-            }
-
-            int spaceUsed = interiorFrame.getBuffer().capacity() - interiorFrame.getTotalFreeSpace();
-            if (spaceUsed + spaceNeeded > interiorMaxBytes) {
-
-                ISplitKey copyKey = splitKey.duplicate(leafFrame.getTupleWriter().createTupleReference());
-                tuple = copyKey.getTuple();
-
-                frontier.lastTuple.resetByTupleIndex(interiorFrame, interiorFrame.getTupleCount() - 1);
-                int splitKeySize = tupleWriter.bytesRequired(frontier.lastTuple, 0, cmp.getKeyFieldCount());
-                splitKey.initData(splitKeySize);
-                tupleWriter.writeTupleFields(frontier.lastTuple, 0, cmp.getKeyFieldCount(),
-                        splitKey.getBuffer().array(), 0);
-                splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
-
-                ((IBTreeInteriorFrame) interiorFrame).deleteGreatest();
-                int finalPageId = freePageManager.takePage(metaFrame);
-                frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), finalPageId));
-                pagesToWrite.add(frontier.page);
-                splitKey.setLeftPage(finalPageId);
-
-                propagateBulk(level + 1, pagesToWrite);
-                frontier.page = bufferCache.confiscatePage(BufferCache.INVALID_DPID);
-                interiorFrame.setPage(frontier.page);
-                interiorFrame.initBuffer((byte) level);
-            }
-            ((IBTreeInteriorFrame) interiorFrame).insertSorted(tuple);
-        }
-
-        private void persistFrontiers(int level, int rightPage) throws HyracksDataException {
-            if (level >= nodeFrontiers.size()) {
-                rootPage = nodeFrontiers.get(level - 1).pageId;
-                releasedLatches = true;
-                return;
-            }
-            if (level < 1) {
-                ICachedPage lastLeaf = nodeFrontiers.get(level).page;
-                int lastLeafPage = nodeFrontiers.get(level).pageId;
-                lastLeaf.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), nodeFrontiers.get(level).pageId));
-                write(lastLeaf);
-                nodeFrontiers.get(level).page = null;
-                persistFrontiers(level + 1, lastLeafPage);
-                return;
-            }
-            NodeFrontier frontier = nodeFrontiers.get(level);
-            interiorFrame.setPage(frontier.page);
-            //just finalize = the layer right above the leaves has correct righthand pointers already
-            if (rightPage < 0) {
-                throw new HyracksDataException(
-                        "Error in index creation. Internal node appears to have no rightmost guide");
-            }
-            ((IBTreeInteriorFrame) interiorFrame).setRightmostChildPageId(rightPage);
-            int finalPageId = freePageManager.takePage(metaFrame);
-            frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), finalPageId));
-            write(frontier.page);
-            frontier.pageId = finalPageId;
-            persistFrontiers(level + 1, finalPageId);
-        }
-
-        @Override
-        public void end() throws HyracksDataException {
-            try {
-                persistFrontiers(0, -1);
-                super.end();
-            } catch (HyracksDataException | RuntimeException e) {
-                handleException();
-                throw e;
-            }
-        }
-
-        @Override
-        public void abort() throws HyracksDataException {
-            super.handleException();
-        }
-
-        private void logState(ITupleReference tuple, Exception e) {
-            try {
-                ObjectNode state = JSONUtil.createObject();
-                state.set("leafFrame", leafFrame.getState());
-                state.set("interiorFrame", interiorFrame.getState());
-                int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
-                        interiorFrame.getBytesRequiredToWriteTuple(tuple));
-                state.put("tupleSize", tupleSize);
-                state.put("spaceNeeded", tupleWriter.bytesRequired(tuple) + slotSize);
-                state.put("spaceUsed", leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace());
-                state.put("leafMaxBytes", leafMaxBytes);
-                state.put("maxTupleSize", maxTupleSize);
-                LOGGER.error("failed to add tuple {}", state, e);
-            } catch (Throwable t) {
-                e.addSuppressed(t);
-            }
-        }
+        return new BTreeNSMBulkLoader(fillFactor, verifyInput, callback, this);
     }
 
     @SuppressWarnings("rawtypes")
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java
new file mode 100644
index 0000000..04c84e1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java
@@ -0,0 +1,279 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.btree.impls;
+
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
+import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.common.api.ISplitKey;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndexBulkLoader;
+import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+import org.apache.hyracks.util.JSONUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class BTreeNSMBulkLoader extends AbstractTreeIndexBulkLoader {
+    private static final Logger LOGGER = LogManager.getLogger();
+    protected final ISplitKey splitKey;
+    protected final boolean verifyInput;
+    private final int maxTupleSize;
+
+    public BTreeNSMBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback, ITreeIndex index)
+            throws HyracksDataException {
+        this(fillFactor, verifyInput, callback, index, index.getLeafFrameFactory().createFrame());
+    }
+
+    protected BTreeNSMBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback, ITreeIndex index,
+            ITreeIndexFrame leafFrame) throws HyracksDataException {
+        super(fillFactor, callback, index, leafFrame);
+        this.verifyInput = verifyInput;
+        splitKey = new BTreeSplitKey(tupleWriter.createTupleReference());
+        splitKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
+        maxTupleSize = ((BTree) index).maxTupleSize;
+    }
+
+    @Override
+    public void add(ITupleReference tuple) throws HyracksDataException {
+        try {
+            int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
+                    interiorFrame.getBytesRequiredToWriteTuple(tuple));
+            NodeFrontier leafFrontier = nodeFrontiers.get(0);
+            int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
+            int spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
+
+            // try to free space by compression
+            if (spaceUsed + spaceNeeded > leafMaxBytes) {
+                leafFrame.compress();
+                spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
+            }
+            //full, allocate new page
+            if (spaceUsed + spaceNeeded > leafMaxBytes) {
+                if (leafFrame.getTupleCount() == 0) {
+                    //The current page is empty. Return it.
+                    bufferCache.returnPage(leafFrontier.page, false);
+                } else {
+                    leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+                    if (verifyInput) {
+                        verifyInputTuple(tuple, leafFrontier.lastTuple);
+                    }
+                    //The current page is not empty. Write it.
+                    writeFullLeafPage();
+                }
+                if (tupleSize > maxTupleSize) {
+                    //We need a large page
+                    final long dpid = BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId);
+                    // calculate required number of pages.
+                    int headerSize = Math.max(leafFrame.getPageHeaderSize(), interiorFrame.getPageHeaderSize());
+                    final int multiplier =
+                            (int) Math.ceil((double) tupleSize / (bufferCache.getPageSize() - headerSize));
+                    if (multiplier > 1) {
+                        leafFrontier.page = bufferCache.confiscateLargePage(dpid, multiplier,
+                                freePageManager.takeBlock(metaFrame, multiplier - 1));
+                    } else {
+                        leafFrontier.page = bufferCache.confiscatePage(dpid);
+                    }
+                    leafFrame.setPage(leafFrontier.page);
+                    leafFrame.initBuffer((byte) 0);
+                    ((IBTreeLeafFrame) leafFrame).setLargeFlag(true);
+                } else {
+                    //allocate a new page
+                    confiscateNewLeafPage();
+                }
+            } else {
+                if (verifyInput && leafFrame.getTupleCount() > 0) {
+                    leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+                    verifyInputTuple(tuple, leafFrontier.lastTuple);
+                }
+            }
+            ((IBTreeLeafFrame) leafFrame).insertSorted(tuple);
+        } catch (HyracksDataException | RuntimeException e) {
+            logState(tuple, e);
+            handleException();
+            throw e;
+        }
+    }
+
+    protected void verifyInputTuple(ITupleReference tuple, ITupleReference prevTuple) throws HyracksDataException {
+        // New tuple should be strictly greater than last tuple.
+        int cmpResult = cmp.compare(tuple, prevTuple);
+        if (cmpResult < 0) {
+            throw HyracksDataException.create(ErrorCode.UNSORTED_LOAD_INPUT);
+        }
+        if (cmpResult == 0) {
+            throw HyracksDataException.create(ErrorCode.DUPLICATE_LOAD_INPUT);
+        }
+    }
+
+    protected void propagateBulk(int level, List<ICachedPage> pagesToWrite) throws HyracksDataException {
+        if (splitKey.getBuffer() == null) {
+            return;
+        }
+
+        if (level >= nodeFrontiers.size()) {
+            addLevel();
+        }
+
+        NodeFrontier frontier = nodeFrontiers.get(level);
+        interiorFrame.setPage(frontier.page);
+
+        ITupleReference tuple = splitKey.getTuple();
+        int tupleBytes = tupleWriter.bytesRequired(tuple, 0, cmp.getKeyFieldCount());
+        int spaceNeeded = tupleBytes + slotSize + 4;
+        if (tupleBytes > interiorFrame.getMaxTupleSize(bufferCache.getPageSize())) {
+            throw HyracksDataException.create(ErrorCode.RECORD_IS_TOO_LARGE, tupleBytes,
+                    interiorFrame.getMaxTupleSize(bufferCache.getPageSize()));
+        }
+
+        int spaceUsed = interiorFrame.getBuffer().capacity() - interiorFrame.getTotalFreeSpace();
+        if (spaceUsed + spaceNeeded > interiorMaxBytes) {
+            ISplitKey copyKey = splitKey.duplicate(tupleWriter.createTupleReference());
+            tuple = copyKey.getTuple();
+
+            frontier.lastTuple.resetByTupleIndex(interiorFrame, interiorFrame.getTupleCount() - 1);
+            int splitKeySize = tupleWriter.bytesRequired(frontier.lastTuple, 0, cmp.getKeyFieldCount());
+            splitKey.initData(splitKeySize);
+            tupleWriter.writeTupleFields(frontier.lastTuple, 0, cmp.getKeyFieldCount(), splitKey.getBuffer().array(),
+                    0);
+            splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+
+            ((IBTreeInteriorFrame) interiorFrame).deleteGreatest();
+            int finalPageId = freePageManager.takePage(metaFrame);
+            frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, finalPageId));
+            pagesToWrite.add(frontier.page);
+            splitKey.setLeftPage(finalPageId);
+
+            propagateBulk(level + 1, pagesToWrite);
+            frontier.page = bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
+            interiorFrame.setPage(frontier.page);
+            interiorFrame.initBuffer((byte) level);
+        }
+        ((IBTreeInteriorFrame) interiorFrame).insertSorted(tuple);
+    }
+
+    private void persistFrontiers(int level, int rightPage) throws HyracksDataException {
+        if (level >= nodeFrontiers.size()) {
+            setRootPageId(nodeFrontiers.get(level - 1).pageId);
+            releasedLatches = true;
+            return;
+        }
+        if (level < 1) {
+            ICachedPage lastLeaf = nodeFrontiers.get(level).page;
+            int lastLeafPage = nodeFrontiers.get(level).pageId;
+            lastLeaf.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, nodeFrontiers.get(level).pageId));
+            writeLastLeaf(lastLeaf);
+            nodeFrontiers.get(level).page = null;
+            persistFrontiers(level + 1, lastLeafPage);
+            return;
+        }
+        NodeFrontier frontier = nodeFrontiers.get(level);
+        interiorFrame.setPage(frontier.page);
+        //just finalize = the layer right above the leaves has correct righthand pointers already
+        if (rightPage < 0) {
+            throw new HyracksDataException("Error in index creation. Internal node appears to have no rightmost guide");
+        }
+        ((IBTreeInteriorFrame) interiorFrame).setRightmostChildPageId(rightPage);
+        int finalPageId = freePageManager.takePage(metaFrame);
+        frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, finalPageId));
+        write(frontier.page);
+        frontier.pageId = finalPageId;
+        persistFrontiers(level + 1, finalPageId);
+    }
+
+    @Override
+    public void end() throws HyracksDataException {
+        try {
+            persistFrontiers(0, -1);
+            super.end();
+        } catch (HyracksDataException | RuntimeException e) {
+            handleException();
+            throw e;
+        }
+    }
+
+    @Override
+    public void abort() throws HyracksDataException {
+        super.handleException();
+    }
+
+    protected void writeFullLeafPage() throws HyracksDataException {
+        final NodeFrontier leafFrontier = nodeFrontiers.get(0);
+        leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+        final int splitKeySize = tupleWriter.bytesRequired(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount());
+        splitKey.initData(splitKeySize);
+        tupleWriter.writeTupleFields(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount(), splitKey.getBuffer().array(),
+                0);
+        splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+        splitKey.setLeftPage(leafFrontier.pageId);
+
+        propagateBulk(1, pagesToWrite);
+
+        leafFrontier.pageId = freePageManager.takePage(metaFrame);
+
+        ((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId);
+
+        write(leafFrontier.page);
+        for (ICachedPage c : pagesToWrite) {
+            write(c);
+        }
+        pagesToWrite.clear();
+        splitKey.setRightPage(leafFrontier.pageId);
+    }
+
+    protected void writeLastLeaf(ICachedPage page) throws HyracksDataException {
+        write(page);
+    }
+
+    protected final void confiscateNewLeafPage() throws HyracksDataException {
+        final NodeFrontier leafFrontier = nodeFrontiers.get(0);
+        final long dpid = BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId);
+        leafFrontier.page = bufferCache.confiscatePage(dpid);
+        leafFrame.setPage(leafFrontier.page);
+        leafFrame.initBuffer((byte) 0);
+    }
+
+    private void logState(ITupleReference tuple, Exception e) {
+        try {
+            ObjectNode state = JSONUtil.createObject();
+            state.set("leafFrame", leafFrame.getState());
+            state.set("interiorFrame", interiorFrame.getState());
+            int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
+                    interiorFrame.getBytesRequiredToWriteTuple(tuple));
+            state.put("tupleSize", tupleSize);
+            state.put("spaceNeeded", tupleWriter.bytesRequired(tuple) + slotSize);
+            state.put("spaceUsed", leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace());
+            state.put("leafMaxBytes", leafMaxBytes);
+            state.put("maxTupleSize", maxTupleSize);
+            LOGGER.error("failed to add tuple {}", state, e);
+        } catch (Throwable t) {
+            e.addSuppressed(t);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
index ae6bbaa..179f1da 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
@@ -24,8 +24,8 @@
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.util.HyracksConstants;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.btree.api.IBTreeFrame;
 import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.btree.api.IDiskBTreeStatefulPointSearchCursor;
 import org.apache.hyracks.storage.am.common.api.IPageManager;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
@@ -38,7 +38,6 @@
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
 import org.apache.hyracks.storage.common.NoOpIndexCursorStats;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
@@ -76,7 +75,8 @@
     private void search(ITreeIndexCursor cursor, ISearchPredicate searchPred, BTreeOpContext ctx)
             throws HyracksDataException {
         ctx.reset();
-        ctx.setPred((RangePredicate) searchPred);
+        RangePredicate rangePredicate = (RangePredicate) searchPred;
+        ctx.setPred(rangePredicate);
         ctx.setCursor(cursor);
         if (ctx.getPred().getLowKeyComparator() == null) {
             ctx.getPred().setLowKeyComparator(ctx.getCmp());
@@ -87,25 +87,17 @@
         cursor.setBufferCache(bufferCache);
         cursor.setFileId(getFileId());
 
-        if (cursor instanceof DiskBTreePointSearchCursor) {
-            DiskBTreePointSearchCursor pointCursor = (DiskBTreePointSearchCursor) cursor;
+        if (cursor instanceof IDiskBTreeStatefulPointSearchCursor) {
+            IDiskBTreeStatefulPointSearchCursor pointCursor = (IDiskBTreeStatefulPointSearchCursor) cursor;
             int lastPageId = pointCursor.getLastPageId();
-            if (lastPageId != BufferCache.INVALID_PAGEID) {
-                // check whether the last leaf page contains this key
-                ICachedPage lastPage =
-                        bufferCache.pin(BufferedFileHandle.getDiskPageId(getFileId(), lastPageId), false);
-                ctx.getLeafFrame().setPage(lastPage);
-                if (fitInPage(ctx.getPred().getLowKey(), ctx.getPred().getLowKeyComparator(), ctx.getLeafFrame())) {
-                    // use this page
-                    ctx.getCursorInitialState().setPage(lastPage);
-                    ctx.getCursorInitialState().setPageId(lastPageId);
-                    pointCursor.open(ctx.getCursorInitialState(), searchPred);
+            if (lastPageId != IBufferCache.INVALID_PAGEID) {
+                if (fitInPage(ctx.getPred().getLowKey(), ctx.getPred().getLowKeyComparator(), pointCursor.getFrame())) {
+                    pointCursor.setCursorToNextKey(searchPred);
                     return;
                 } else {
-                    // release the last page and clear the states of this cursor
+                    // release the last page, clear the states of this cursor, and close the cursor
                     // then retry the search from root to leaf
-                    bufferCache.unpin(lastPage);
-                    pointCursor.clearSearchState();
+                    cursor.close();
                 }
             }
         }
@@ -113,7 +105,7 @@
         searchDown(rootNode, rootPage, ctx, cursor);
     }
 
-    private boolean fitInPage(ITupleReference key, MultiComparator comparator, IBTreeFrame frame)
+    private boolean fitInPage(ITupleReference key, MultiComparator comparator, ITreeIndexFrame frame)
             throws HyracksDataException {
         // assume that search keys are sorted (non-decreasing)
         ITupleReference rightmostTuple = frame.getRightmostTuple();
@@ -191,14 +183,14 @@
         }
 
         @Override
-        public DiskBTreeRangeSearchCursor createSearchCursor(boolean exclusive) {
+        public ITreeIndexCursor createSearchCursor(boolean exclusive) {
             IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
             return new DiskBTreeRangeSearchCursor(leafFrame, exclusive, (IIndexCursorStats) iap.getParameters()
                     .getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE));
         }
 
         @Override
-        public BTreeRangeSearchCursor createPointCursor(boolean exclusive, boolean stateful) {
+        public ITreeIndexCursor createPointCursor(boolean exclusive, boolean stateful) {
             IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
             return new DiskBTreePointSearchCursor(leafFrame, exclusive, stateful);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
index 1bf3ecf..8fd9a96 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
@@ -21,13 +21,16 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.btree.api.IDiskBTreeStatefulPointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
 import org.apache.hyracks.storage.am.common.ophelpers.FindTupleMode;
 import org.apache.hyracks.storage.am.common.ophelpers.FindTupleNoExactMatchPolicy;
 import org.apache.hyracks.storage.common.ICursorInitialState;
 import org.apache.hyracks.storage.common.ISearchPredicate;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 
-public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor {
+public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor
+        implements IDiskBTreeStatefulPointSearchCursor {
     /**
      * A stateful cursor keeps the search state (last search page Id + index) across multiple searches
      * until {@link #clearSearchState()} is called explicity
@@ -36,16 +39,14 @@
 
     private boolean nextHasBeenCalled;
 
-    private int lastPageId = BufferCache.INVALID_PAGEID;
+    private int lastPageId;
     private int lastTupleIndex = 0;
 
     public DiskBTreePointSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes, boolean stateful) {
         super(frame, exclusiveLatchNodes);
         this.stateful = stateful;
-    }
-
-    public DiskBTreePointSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes) {
-        this(frame, exclusiveLatchNodes, false);
+        lastPageId = IBufferCache.INVALID_PAGEID;
+        lastTupleIndex = 0;
     }
 
     @Override
@@ -59,6 +60,12 @@
     }
 
     @Override
+    public void doClose() throws HyracksDataException {
+        clearSearchState();
+        super.doClose();
+    }
+
+    @Override
     public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         // in case open is called multiple times without closing
         if (page != null) {
@@ -71,30 +78,10 @@
         page = initialState.getPage();
         isPageDirty = false;
         frame.setPage(page);
-
-        pred = (RangePredicate) searchPred;
-        lowKeyCmp = pred.getLowKeyComparator();
-        lowKey = pred.getLowKey();
-
-        reusablePredicate.setLowKeyComparator(originalKeyCmp);
-
-        lowKeyFtm = FindTupleMode.EXACT;
-        lowKeyFtp = FindTupleNoExactMatchPolicy.NONE;
-
-        nextHasBeenCalled = false;
-
-        // only get the low key position
-        tupleIndex = getLowKeyIndex();
-        if (stateful) {
-            lastPageId = pageId;
-            if (tupleIndex >= 0) {
-                lastTupleIndex = tupleIndex;
-            } else {
-                lastTupleIndex = -tupleIndex - 1;
-            }
-        }
+        setCursorToNextKey(searchPred);
     }
 
+    @Override
     public int getLastPageId() {
         return lastPageId;
     }
@@ -108,9 +95,38 @@
         }
     }
 
-    public void clearSearchState() {
-        this.lastPageId = BufferCache.INVALID_PAGEID;
+    @Override
+    public void setCursorToNextKey(ISearchPredicate searchPred) throws HyracksDataException {
+        pred = (RangePredicate) searchPred;
+        lowKeyCmp = pred.getLowKeyComparator();
+        lowKey = pred.getLowKey();
+
+        reusablePredicate.setLowKeyComparator(originalKeyCmp);
+
+        lowKeyFtm = FindTupleMode.EXACT;
+        lowKeyFtp = FindTupleNoExactMatchPolicy.NONE;
+
+        nextHasBeenCalled = false;
+
+        // only get the lowKey position
+        tupleIndex = getLowKeyIndex();
+        if (stateful) {
+            lastPageId = pageId;
+            if (tupleIndex >= 0) {
+                lastTupleIndex = tupleIndex;
+            } else {
+                lastTupleIndex = -tupleIndex - 1;
+            }
+        }
+    }
+
+    private void clearSearchState() {
+        this.lastPageId = IBufferCache.INVALID_PAGEID;
         this.lastTupleIndex = 0;
     }
 
+    @Override
+    public ITreeIndexFrame getFrame() {
+        return frame;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index eab8c96..1fe91d9 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
@@ -44,6 +44,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,6 +55,8 @@
 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;
@@ -107,6 +110,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 +120,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 +129,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;
@@ -162,6 +167,8 @@
         if (this.tupleFilterFactory != null && this.retainMissing) {
             throw new IllegalStateException("RetainMissing with tuple filter is not supported");
         }
+
+        tupleProjector = projectorFactory.createTupleProjector(ctx);
     }
 
     protected abstract ISearchPredicate createSearchPredicate();
@@ -349,10 +356,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;
         }
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-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
index e16baf9..ee4c4f7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
@@ -56,19 +56,21 @@
             ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             IMetadataPageManagerFactory metadataPageManagerFactory,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter,
-            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
+            boolean isSecondaryNoIncrementalMaintenance) {
         super(typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, path,
                 storageManager, mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories,
                 btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
                 metadataPageManagerFactory, null, ioSchedulerProvider, durable,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector);
+                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector,
+                isSecondaryNoIncrementalMaintenance);
     }
 
     private ExternalBTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
-            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter)
-            throws HyracksDataException {
+            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
+            boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
         super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter);
+                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
     }
 
     @Override
@@ -97,6 +99,6 @@
         LSMBTreeLocalResource lsmBtree = (LSMBTreeLocalResource) LSMBTreeLocalResource.fromJson(registry, json);
         return new ExternalBTreeLocalResource(registry, json, lsmBtree.bloomFilterKeyFields,
                 lsmBtree.bloomFilterFalsePositiveRate, lsmBtree.isPrimary, lsmBtree.btreeFields,
-                lsmBtree.hasBloomFilter);
+                lsmBtree.hasBloomFilter, lsmBtree.isSecondaryNoIncrementalMaintenance);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
index e4c29ba..3a28057 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
@@ -46,13 +46,13 @@
             IMetadataPageManagerFactory metadataPageManagerFactory,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
             Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
-            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
-            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+            double bloomFilterFalsePositiveRate, int[] btreeFields, boolean hasBloomFilter, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
                 ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, bloomFilterKeyFields,
-                bloomFilterFalsePositiveRate, isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
-                hasBloomFilter, nullTypeTraits, nullIntrospector);
+                bloomFilterFalsePositiveRate, false, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
+                hasBloomFilter, nullTypeTraits, nullIntrospector, true);
     }
 
     @Override
@@ -61,6 +61,7 @@
                 bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
                 mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, bloomFilterKeyFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector);
+                ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector,
+                isSecondaryNoIncrementalMaintenance);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
index 863ae56..f2c5263 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
@@ -59,19 +59,21 @@
             ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             IMetadataPageManagerFactory metadataPageManagerFactory,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter,
-            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
+            boolean isSecondaryNoIncrementalMaintenance) {
         super(typeTraits, cmpFactories, buddyBtreeFields, bloomFilterFalsePositiveRate, isPrimary, path, storageManager,
                 mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields,
                 filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
                 metadataPageManagerFactory, null, ioSchedulerProvider, durable,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector);
+                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector,
+                isSecondaryNoIncrementalMaintenance);
     }
 
     private ExternalBTreeWithBuddyLocalResource(IPersistedResourceRegistry registry, JsonNode json,
             int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
-            boolean hasBloomFilter) throws HyracksDataException {
+            boolean hasBloomFilter, boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
         super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter);
+                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
     }
 
     @Override
@@ -100,6 +102,6 @@
         LSMBTreeLocalResource lsmBtree = (LSMBTreeLocalResource) LSMBTreeLocalResource.fromJson(registry, json);
         return new ExternalBTreeWithBuddyLocalResource(registry, json, lsmBtree.bloomFilterKeyFields,
                 lsmBtree.bloomFilterFalsePositiveRate, lsmBtree.isPrimary, lsmBtree.btreeFields,
-                lsmBtree.hasBloomFilter);
+                lsmBtree.hasBloomFilter, lsmBtree.isSecondaryNoIncrementalMaintenance);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
index 39c8aed..995a23b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
@@ -46,13 +46,13 @@
             IMetadataPageManagerFactory metadataPageManagerFactory,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
             Map<String, String> mergePolicyProperties, boolean durable, int[] buddyBtreeFields,
-            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
-            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+            double bloomFilterFalsePositiveRate, int[] btreeFields, boolean hasBloomFilter, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
                 ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, buddyBtreeFields,
-                bloomFilterFalsePositiveRate, isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
-                hasBloomFilter, nullTypeTraits, nullIntrospector);
+                bloomFilterFalsePositiveRate, false, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
+                hasBloomFilter, nullTypeTraits, nullIntrospector, true);
     }
 
     @Override
@@ -61,6 +61,7 @@
                 bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
                 mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector);
+                ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector,
+                isSecondaryNoIncrementalMaintenance);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
index 3800d17..3e03e5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.common.projection.ITupleProjectorFactory;
 
 public class LSMBTreeBatchPointSearchOperatorDescriptor extends BTreeSearchOperatorDescriptor {
     private static final long serialVersionUID = 1L;
@@ -38,10 +39,11 @@
             IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, ITupleFilterFactory tupleFilterFactory,
-            long outputLimit) {
+            long outputLimit, ITupleProjectorFactory tupleProjectorFactory) {
         super(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
-                maxFilterFieldIndexes, false, null, tupleFilterFactory, outputLimit, false, null, null);
+                maxFilterFieldIndexes, false, null, tupleFilterFactory, outputLimit, false, null, null,
+                tupleProjectorFactory);
     }
 
     @Override
@@ -51,7 +53,7 @@
                 recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), lowKeyFields, highKeyFields,
                 lowKeyInclusive, highKeyInclusive, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, tupleFilterFactory,
-                outputLimit);
+                outputLimit, tupleProjectorFactory);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
index 30813ef..9b8c353 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
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
index 656ae02..a7e433c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
@@ -59,6 +59,7 @@
     protected final boolean isPrimary;
     protected final int[] btreeFields;
     protected final ICompressorDecompressorFactory compressorDecompressorFactory;
+    protected final boolean isSecondaryNoIncrementalMaintenance;
 
     public LSMBTreeLocalResource(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
             int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, boolean isPrimary, String path,
@@ -70,7 +71,8 @@
             IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable,
             ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter,
-            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
+            boolean isSecondaryNoIncrementalMaintenance) {
         super(path, storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
                 vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
@@ -81,12 +83,13 @@
         this.btreeFields = btreeFields;
         this.compressorDecompressorFactory = compressorDecompressorFactory;
         this.hasBloomFilter = hasBloomFilter;
+        this.isSecondaryNoIncrementalMaintenance = isSecondaryNoIncrementalMaintenance;
     }
 
     protected LSMBTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
             double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
-            ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter)
-            throws HyracksDataException {
+            ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter,
+            boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
         super(registry, json);
         this.bloomFilterKeyFields = bloomFilterKeyFields;
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
@@ -94,6 +97,7 @@
         this.btreeFields = btreeFields;
         this.compressorDecompressorFactory = compressorDecompressorFactory;
         this.hasBloomFilter = hasBloomFilter;
+        this.isSecondaryNoIncrementalMaintenance = isSecondaryNoIncrementalMaintenance;
     }
 
     @Override
@@ -114,6 +118,10 @@
                 compressorDecompressorFactory, hasBloomFilter, nullTypeTraits, nullIntrospector);
     }
 
+    public boolean isSecondaryNoIncrementalMaintenance() {
+        return isSecondaryNoIncrementalMaintenance;
+    }
+
     @Override
     public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
         final ObjectNode jsonObject = registry.getClassIdentifier(getClass(), serialVersionUID);
@@ -131,8 +139,10 @@
         final JsonNode compressorDecompressorNode = json.get("compressorDecompressorFactory");
         final ICompressorDecompressorFactory compDecompFactory = (ICompressorDecompressorFactory) registry
                 .deserializeOrDefault(compressorDecompressorNode, NoOpCompressorDecompressorFactory.class);
+        boolean isSecondaryNoIncrementalMaintenance =
+                getOrDefaultBoolean(json, "isSecondaryNoIncrementalMaintenance", false);
         return new LSMBTreeLocalResource(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary,
-                btreeFields, compDecompFactory, hasBloomFilter);
+                btreeFields, compDecompFactory, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
     }
 
     @Override
@@ -145,13 +155,16 @@
         json.put("isPrimary", isPrimary);
         json.putPOJO("btreeFields", btreeFields);
         json.putPOJO("compressorDecompressorFactory", compressorDecompressorFactory.toJson(registry));
+        json.put("isSecondaryNoIncrementalMaintenance", isSecondaryNoIncrementalMaintenance);
     }
 
     private static boolean getOrDefaultHasBloomFilter(JsonNode json, boolean isPrimary) {
-        if (json.has(HAS_BLOOM_FILTER_FIELD)) {
-            return json.get(HAS_BLOOM_FILTER_FIELD).asBoolean();
-        }
-        // for backward compatibiliy, only primary indexes have bloom filters
-        return isPrimary;
+        // for backward compatibility, only primary indexes have bloom filters
+        return getOrDefaultBoolean(json, HAS_BLOOM_FILTER_FIELD, isPrimary);
     }
+
+    private static boolean getOrDefaultBoolean(JsonNode jsonNode, String fieldName, boolean defaultValue) {
+        return jsonNode.has(fieldName) ? jsonNode.get(fieldName).asBoolean() : defaultValue;
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
index e926a48..6695e90 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
@@ -48,6 +48,7 @@
     protected final boolean isPrimary;
     protected final int[] btreeFields;
     protected final ICompressorDecompressorFactory compressorDecompressorFactory;
+    protected final boolean isSecondaryNoIncrementalMaintenance;
 
     public LSMBTreeLocalResourceFactory(IStorageManager storageManager, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
@@ -59,7 +60,8 @@
             Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
             double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
             ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter,
-            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector,
+            boolean isSecondaryNoIncrementalMaintenance) {
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
                 vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
@@ -70,6 +72,7 @@
         this.isPrimary = isPrimary;
         this.btreeFields = btreeFields;
         this.compressorDecompressorFactory = compressorDecompressorFactory;
+        this.isSecondaryNoIncrementalMaintenance = isSecondaryNoIncrementalMaintenance;
     }
 
     @Override
@@ -78,7 +81,8 @@
                 isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory, mergePolicyProperties,
                 filterTypeTraits, filterCmpFactories, btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory,
                 pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, durable,
-                compressorDecompressorFactory, hasBloomFilter, nullTypeTraits, nullIntrospector);
+                compressorDecompressorFactory, hasBloomFilter, nullTypeTraits, nullIntrospector,
+                isSecondaryNoIncrementalMaintenance);
     }
 
     private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
index c3d1416..d2fbbef 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -341,7 +341,7 @@
                 try {
                     List<ILSMComponent> mergedComponents = mergeOp.getMergingComponents();
                     long numElements = getNumberOfElements(mergedComponents);
-                    mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(), null,
+                    mergedComponent = createDiskComponent(getMergeComponentFactory(), mergeOp.getTarget(), null,
                             mergeOp.getBloomFilterTarget(), true);
                     IPageWriteCallback pageWriteCallback = pageWriteCallbackFactory.createPageWriteCallback();
                     componentBulkLoader = mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false,
@@ -418,7 +418,7 @@
     }
 
     public ILSMIndexAccessor createAccessor(AbstractLSMIndexOperationContext opCtx) {
-        return new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
+        return new LSMTreeIndexAccessor(getHarness(), opCtx, getCursorFactory());
     }
 
     @Override
@@ -483,8 +483,28 @@
             returnDeletedTuples = true;
         }
         IIndexCursorStats stats = new IndexCursorStats();
-        LSMBTreeRangeSearchCursor cursor = new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples, stats);
+        LSMBTreeRangeSearchCursor cursor = createCursor(opCtx, returnDeletedTuples, stats);
         return new LSMBTreeMergeOperation(accessor, cursor, stats, mergeFileRefs.getInsertIndexFileReference(),
                 mergeFileRefs.getBloomFilterFileReference(), callback, getIndexIdentifier());
     }
+
+    public LSMBTreeBatchPointSearchCursor createBatchPointSearchCursor(ILSMIndexOperationContext opCtx) {
+        return new LSMBTreeBatchPointSearchCursor(opCtx);
+    }
+
+    protected LSMBTreeRangeSearchCursor createCursor(AbstractLSMIndexOperationContext opCtx,
+            boolean returnDeletedTuples, IIndexCursorStats stats) {
+        return new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples, stats);
+    }
+
+    /**
+     * @return Merge component factory (could be different from {@link #componentFactory}
+     */
+    protected ILSMDiskComponentFactory getMergeComponentFactory() {
+        return componentFactory;
+    }
+
+    protected ICursorFactory getCursorFactory() {
+        return cursorFactory;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java
index 8ab6fb1..73d06d2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeBatchPointSearchCursor.java
@@ -21,14 +21,15 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
 import org.apache.hyracks.storage.am.btree.impls.BatchPredicate;
-import org.apache.hyracks.storage.am.btree.impls.DiskBTreePointSearchCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 
 /**
  * This cursor performs point searches for each batch of search keys.
  * Assumption: the search keys must be sorted into the increasing order.
- *
  */
 public class LSMBTreeBatchPointSearchCursor extends LSMBTreePointSearchCursor {
 
@@ -42,7 +43,9 @@
         while (!foundTuple && batchPred.hasNext()) {
             batchPred.next();
             if (foundIn >= 0) {
-                btreeCursors[foundIn].close();
+                if (operationalComponents.get(foundIn).getType() == LSMComponentType.MEMORY) {
+                    btreeCursors[foundIn].close();
+                }
                 foundIn = -1;
             }
             foundTuple = super.doHasNext();
@@ -56,6 +59,11 @@
     }
 
     @Override
+    protected ITreeIndexCursor createCursor(LSMComponentType type, BTreeAccessor btreeAccessor) {
+        return btreeAccessor.createPointCursor(false, true);
+    }
+
+    @Override
     protected boolean isSearchCandidate(int componentIndex) throws HyracksDataException {
         if (!super.isSearchCandidate(componentIndex)) {
             return false;
@@ -68,21 +76,6 @@
                 maxFileterKey, opCtx.getFilterCmp());
     }
 
-    @Override
-    protected void closeCursors() throws HyracksDataException {
-        super.closeCursors();
-        if (btreeCursors != null) {
-            // clear search states of btree cursors
-            for (int i = 0; i < numBTrees; ++i) {
-                if (btreeCursors[i] != null) {
-                    if (btreeCursors[i] instanceof DiskBTreePointSearchCursor) {
-                        ((DiskBTreePointSearchCursor) btreeCursors[i]).clearSearchState();
-                    }
-                }
-            }
-        }
-    }
-
     public int getKeyIndex() {
         return ((BatchPredicate) predicate).getKeyIndex();
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
index 1312e30..a00e10e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
@@ -43,7 +43,7 @@
 import org.apache.hyracks.storage.common.MultiComparator;
 import org.apache.hyracks.util.trace.ITracer;
 
-public final class LSMBTreeOpContext extends AbstractLSMIndexOperationContext {
+public class LSMBTreeOpContext extends AbstractLSMIndexOperationContext {
 
     /*
      * Finals
@@ -74,9 +74,9 @@
             IBinaryComparatorFactory[] filterCmpFactories, ITracer tracer) {
         super(index, btreeFields, filterFields, filterCmpFactories, searchCallback, modificationCallback, tracer);
         LSMBTreeMemoryComponent c = (LSMBTreeMemoryComponent) mutableComponents.get(0);
-        IBinaryComparatorFactory cmpFactories[] = c.getIndex().getComparatorFactories();
+        IBinaryComparatorFactory[] cmpFactories = c.getIndex().getComparatorFactories();
         if (cmpFactories[0] != null) {
-            this.cmp = MultiComparator.create(c.getIndex().getComparatorFactories());
+            this.cmp = createMultiComparator(c.getIndex().getComparatorFactories());
         } else {
             this.cmp = null;
         }
@@ -112,6 +112,10 @@
         insertSearchCursor = new LSMBTreePointSearchCursor(this);
     }
 
+    protected MultiComparator createMultiComparator(IBinaryComparatorFactory[] cmpFactories) {
+        return MultiComparator.create(cmpFactories);
+    }
+
     @Override
     public void setOperation(IndexOperation newOp) {
         reset();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
index d4903d9..9740bc4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
@@ -199,7 +199,8 @@
         for (int i = 0; i < numBTrees; i++) {
             ILSMComponent component = operationalComponents.get(i);
             BTree btree = (BTree) component.getIndex();
-            if (component.getType() == LSMComponentType.MEMORY) {
+            LSMComponentType type = component.getType();
+            if (type == LSMComponentType.MEMORY) {
                 includeMutableComponent = true;
                 if (bloomFilters[i] != null) {
                     destroyAndNullifyCursorAtIndex(i);
@@ -212,8 +213,8 @@
             }
 
             if (btreeAccessors[i] == null) {
-                btreeAccessors[i] = btree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-                btreeCursors[i] = btreeAccessors[i].createPointCursor(false, false);
+                btreeAccessors[i] = createAccessor(type, btree, i);
+                btreeCursors[i] = createCursor(type, btreeAccessors[i]);
             } else {
                 // re-use
                 btreeAccessors[i].reset(btree, NoOpIndexAccessParameters.INSTANCE);
@@ -225,6 +226,14 @@
         hashComputed = false;
     }
 
+    protected BTreeAccessor createAccessor(LSMComponentType type, BTree btree, int i) throws HyracksDataException {
+        return btree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+    }
+
+    protected ITreeIndexCursor createCursor(LSMComponentType type, BTreeAccessor btreeAccessor) {
+        return btreeAccessor.createPointCursor(false, false);
+    }
+
     private void destroyAndNullifyCursorAtIndex(int i) throws HyracksDataException {
         // component at location i was a disk component before, and is now a memory component, or vise versa
         bloomFilters[i] = null;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
index 2c5fb50..968416c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
@@ -49,7 +49,7 @@
     private final RangePredicate reusablePred;
     private ISearchOperationCallback searchCallback;
     private BTreeAccessor[] btreeAccessors;
-    private boolean[] isMemoryComponent;
+    protected boolean[] isMemoryComponent;
     private ArrayTupleBuilder tupleBuilder;
     private boolean canCallProceed = true;
     private boolean resultOfSearchCallbackProceed = false;
@@ -149,6 +149,7 @@
                                 // There are no more elements in the memory component.. can safely skip locking for the
                                 // remaining operations
                                 includeMutableComponent = false;
+                                excludeMemoryComponent();
                             }
                         }
                     } else {
@@ -180,6 +181,7 @@
                         // the tree of head tuple
                         // the head element of PQ is useless now
                         PriorityQueueElement e = outputPriorityQueue.poll();
+                        markAsDeleted(e);
                         pushIntoQueueFromCursorAndReplaceThisElement(e);
                     } else {
                         // If the previous tuple and the head tuple are different
@@ -200,6 +202,14 @@
 
     }
 
+    protected void excludeMemoryComponent() {
+        //NoOp
+    }
+
+    protected void markAsDeleted(PriorityQueueElement e) throws HyracksDataException {
+        //NoOp
+    }
+
     private void pushOutputElementIntoQueueIfNeeded() throws HyracksDataException {
         if (needPushElementIntoQueue) {
             pushIntoQueueFromCursorAndReplaceThisElement(outputElement);
@@ -250,7 +260,7 @@
         }
     }
 
-    private int replaceFrom() throws HyracksDataException {
+    protected int replaceFrom() throws HyracksDataException {
         int replaceFrom = -1;
         if (!switchPossible) {
             return replaceFrom;
@@ -386,20 +396,21 @@
         }
         for (int i = 0; i < numBTrees; i++) {
             ILSMComponent component = operationalComponents.get(i);
+            LSMComponentType type = component.getType();
             BTree btree;
             if (component.getType() == LSMComponentType.MEMORY) {
                 includeMutableComponent = true;
             }
             btree = (BTree) component.getIndex();
             if (btreeAccessors[i] == null || destroyIncompatible(component, i)) {
-                btreeAccessors[i] = btree.createAccessor(iap);
-                rangeCursors[i] = btreeAccessors[i].createSearchCursor(false);
+                btreeAccessors[i] = createAccessor(type, btree, i);
+                rangeCursors[i] = createCursor(type, btreeAccessors[i]);
             } else {
                 // re-use
                 btreeAccessors[i].reset(btree, iap);
                 rangeCursors[i].close();
             }
-            isMemoryComponent[i] = component.getType() == LSMComponentType.MEMORY;
+            isMemoryComponent[i] = type == LSMComponentType.MEMORY;
         }
         IndexCursorUtils.open(btreeAccessors, rangeCursors, searchPred);
         try {
@@ -433,4 +444,12 @@
         return resultOfSearchCallbackProceed;
     }
 
+    protected BTreeAccessor createAccessor(LSMComponentType type, BTree btree, int index) throws HyracksDataException {
+        return btree.createAccessor(iap);
+    }
+
+    protected IIndexCursor createCursor(LSMComponentType type, BTreeAccessor accessor) {
+        return accessor.createSearchCursor(false);
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
index efacad1..aa72267 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
@@ -46,6 +46,13 @@
         scanCursor = new LSMBTreeDiskComponentScanCursor(opCtx);
     }
 
+    protected LSMBTreeSearchCursor(LSMBTreePointSearchCursor pointCursor, LSMBTreeRangeSearchCursor rangeCursor,
+            LSMBTreeDiskComponentScanCursor scanCursor) {
+        this.pointCursor = pointCursor;
+        this.rangeCursor = rangeCursor;
+        this.scanCursor = scanCursor;
+    }
+
     @Override
     public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         LSMBTreeCursorInitialState lsmInitialState = (LSMBTreeCursorInitialState) initialState;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
index 13a0e27..acb84e1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
@@ -20,7 +20,7 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex.AbstractTreeIndexBulkLoader;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndexBulkLoader;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleWriter;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-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/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
index 6095b26..8ca1a82 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
@@ -16,9 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.hyracks.storage.common.projection;
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+import java.io.DataOutput;
+import java.io.IOException;
 
-select value v from range(1,2) v where v > ?;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public interface ITupleProjector {
+    void project(ITupleReference tuple, DataOutput dos, ArrayTupleBuilder tb) throws IOException;
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
similarity index 61%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
copy to hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
index 6095b26..ff9ecf9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
@@ -16,9 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.hyracks.storage.common.projection;
 
--- param compile-only:string=true
--- param logical-plan:string=true
--- param plan-format:string=json
+import java.io.Serializable;
 
-select value v from range(1,2) v where v > ?;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Tuple projector allows the data source to project the values needed before it pushed up
+ * to the upper operator.
+ */
+public interface ITupleProjectorFactory extends Serializable {
+    ITupleProjector createTupleProjector(IHyracksTaskContext context) throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
index dcb85f6..c73ebee 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
@@ -49,15 +49,17 @@
 public class TestTaskContext implements IHyracksTaskContext {
     private final TestJobletContext jobletContext;
     private final TaskAttemptId taskId;
+    private int partitionCount;
     private WorkspaceFileFactory fileFactory;
     private Map<Object, IStateObject> stateObjectMap = new HashMap<>();
     private Object sharedObject;
     private final IStatsCollector statsCollector = new StatsCollector();
     private final ThreadStats threadStats = new ThreadStats();
 
-    public TestTaskContext(TestJobletContext jobletContext, TaskAttemptId taskId) {
+    public TestTaskContext(TestJobletContext jobletContext, TaskAttemptId taskId, int partitionCount) {
         this.jobletContext = jobletContext;
         this.taskId = taskId;
+        this.partitionCount = partitionCount;
         fileFactory = new WorkspaceFileFactory(this, getIoManager());
     }
 
@@ -129,6 +131,11 @@
     }
 
     @Override
+    public int getPartitionCount() {
+        return partitionCount;
+    }
+
+    @Override
     public synchronized void setStateObject(IStateObject taskState) {
         stateObjectMap.put(taskState.getId(), taskState);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
index 3f78234..2348a1a 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
@@ -38,7 +38,7 @@
 import org.apache.hyracks.api.dataflow.TaskId;
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.api.exceptions.NoOpWarningCollector;
 import org.apache.hyracks.api.io.IODeviceHandle;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.util.CleanupUtils;
@@ -52,22 +52,7 @@
 public class TestUtils {
 
     private static final int DEFAULT_FRAME_SIZE = 32768;
-    public static final IWarningCollector NOOP_WARNING_COLLECTOR = new IWarningCollector() {
-        @Override
-        public void warn(Warning warning) {
-            // no-op
-        }
-
-        @Override
-        public boolean shouldWarn() {
-            return false;
-        }
-
-        @Override
-        public long getTotalWarningsCount() {
-            return 0;
-        }
-    };
+    public static final IWarningCollector NOOP_WARNING_COLLECTOR = NoOpWarningCollector.INSTANCE;
 
     public static IHyracksTaskContext createHyracksTask() {
         return create(DEFAULT_FRAME_SIZE);
@@ -91,7 +76,7 @@
             INCServiceContext serviceCtx = new TestNCServiceContext(ioManager, null);
             TestJobletContext jobletCtx = new TestJobletContext(frameSize, serviceCtx, new JobId(0));
             TaskAttemptId tid = new TaskAttemptId(new TaskId(new ActivityId(new OperatorDescriptorId(0), 0), 0), 0);
-            IHyracksTaskContext taskCtx = new TestTaskContext(jobletCtx, tid);
+            IHyracksTaskContext taskCtx = new TestTaskContext(jobletCtx, tid, 1);
             return taskCtx;
         } catch (HyracksException e) {
             throw new RuntimeException(e);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
index 8e2d6dd..c7d3dff 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
@@ -56,19 +56,21 @@
             ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
-            ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter) {
+            ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter,
+            boolean isSecondaryNoIncrementalMaintenance) {
         super(typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, path,
                 storageManager, mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories,
                 btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
                 metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, durable,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null);
+                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null,
+                isSecondaryNoIncrementalMaintenance);
     }
 
     protected TestLsmBtreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
-            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter)
-            throws HyracksDataException {
+            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
+            boolean isSecondaryNoIncrementalMaintenance) throws HyracksDataException {
         super(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter);
+                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
     }
 
     @Override
@@ -107,7 +109,8 @@
         final boolean isPrimary = json.get("isPrimary").asBoolean();
         final boolean hasBloomFilter = json.get("hasBloomFilter").asBoolean();
         final int[] btreeFields = OBJECT_MAPPER.convertValue(json.get("btreeFields"), int[].class);
+        boolean isSecondaryNoIncrementalMaintenance = json.get("isSecondaryNoIncrementalMaintenance").asBoolean();
         return new TestLsmBtreeLocalResource(registry, json, bloomFilterKeyFields, bloomFilterFalsePositiveRate,
-                isPrimary, btreeFields, hasBloomFilter);
+                isPrimary, btreeFields, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
index 7d9b24a..48adf91 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
@@ -46,12 +46,14 @@
             IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
             Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
-            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter) {
+            double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
+            boolean isSecondaryNoIncrementalMaintenance) {
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
                 vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable,
                 bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
-                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null);
+                NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null,
+                isSecondaryNoIncrementalMaintenance);
     }
 
     @Override
@@ -60,6 +62,6 @@
                 bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
                 mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                vbcProvider, ioSchedulerProvider, durable, hasBloomFilter);
+                vbcProvider, ioSchedulerProvider, durable, hasBloomFilter, isSecondaryNoIncrementalMaintenance);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
index aa9cd5a..144f1ad 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
@@ -740,6 +740,11 @@
         }
 
         @Override
+        public int getPartitionCount() {
+            return 1;
+        }
+
+        @Override
         public ICounterContext getCounterContext() {
             return null;
         }