Merge branch 'gerrit/neo'

Change-Id: I7ba10b2a046866640ab3d646ea6b71aae399f436
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/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/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/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index 10aedc7..5408915 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
@@ -48,7 +48,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;
@@ -390,7 +389,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-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..ce86d24 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
@@ -54,7 +54,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;
@@ -65,25 +64,17 @@
 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.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;
@@ -121,7 +112,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 +123,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 +135,7 @@
         this.astPrintVisitorFactory = compilationProvider.getAstPrintVisitorFactory();
         this.translatorFactory = compilationProvider.getExpressionToPlanTranslatorFactory();
         this.ruleSetFactory = compilationProvider.getRuleSetFactory();
+        this.configurableParameterNames = compilationProvider.getCompilerOptions();
         executionPlans = new ExecutionPlans();
     }
 
@@ -502,10 +475,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/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/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/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/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/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/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/OptimizationConfUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
index d498d69..d94fa04 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;
     }
 
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/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..1bc51f2 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,6 +19,7 @@
 package org.apache.asterix.external.input.record.reader.gcs;
 
 import static org.apache.asterix.external.util.ExternalDataUtils.getIncludeExcludeMatchers;
+import static org.apache.asterix.external.util.google.gcs.GCSUtils.buildClient;
 import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
 
 import java.util.ArrayList;
@@ -67,7 +68,7 @@
         // 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 gcs = buildClient(configuration);
         Storage.BlobListOption options = Storage.BlobListOption.prefix(ExternalDataUtils.getPrefix(configuration));
         Page<Blob> items;
 
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..702ef42 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);
+                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..e1c10ad
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Constants.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.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 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..a88d59b
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/aws/s3/S3Utils.java
@@ -0,0 +1,436 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.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.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 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;
+
+        // 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(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..8a0be99
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSConstants.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+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";
+}
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..553733f
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/google/gcs/GCSUtils.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util.google.gcs;
+
+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.external.util.ExternalDataConstants.KEY_FORMAT;
+import static org.apache.asterix.external.util.ExternalDataUtils.getPrefix;
+import static org.apache.asterix.external.util.ExternalDataUtils.isParquetFormat;
+import static org.apache.asterix.external.util.ExternalDataUtils.validateIncludeExclude;
+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.util.Map;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.util.ExternalDataConstants;
+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.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");
+
+    }
+
+    //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 {
+            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));
+        }
+    }
+}
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-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-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..14d1a6a 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 {
 
@@ -135,7 +136,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..1cdbbea 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
@@ -169,6 +169,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> {
 
@@ -542,7 +543,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());
@@ -601,12 +603,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,
@@ -859,7 +862,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/functions/ExternalFunctionCompilerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
index 5fd96a6..ce98008 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
@@ -174,7 +174,7 @@
         if (fnKind != FunctionKind.SCALAR) {
             return false;
         }
-        if (!(fnInfo instanceof IExternalFunctionInfo)) {
+        if (!fnInfo.isExternal()) {
             return false;
         }
         ExternalFunctionLanguage language = ((IExternalFunctionInfo) fnInfo).getLanguage();
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/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/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java
index 6e1ccef..34ed225 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/functions/IFunctionInfo.java
@@ -38,6 +38,11 @@
         return true;
     }
 
+    default boolean isExternal() {
+        // A function is not external by default.
+        return false;
+    }
+
     /**
      * @param args,
      *            the arguments.
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
index 5fc69b2..56e2e22 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractUnnestMapOperator.java
@@ -25,10 +25,10 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 
 public abstract class AbstractUnnestMapOperator extends AbstractUnnestOperator {
 
-    protected final Mutable<ILogicalExpression> expression;
     protected final List<Object> variableTypes;
     protected boolean propagateInput;
     protected List<Mutable<ILogicalExpression>> additionalFilteringExpressions;
@@ -42,7 +42,6 @@
     public AbstractUnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
             List<Object> variableTypes, boolean propagateInput) {
         super(variables, expression);
-        this.expression = expression;
         this.variableTypes = variableTypes;
         this.propagateInput = propagateInput;
         this.propagateIndexFilter = false;
@@ -90,6 +89,18 @@
         };
     }
 
+    @Override
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+        boolean changed = super.acceptExpressionTransform(visitor);
+
+        if (additionalFilteringExpressions != null) {
+            for (Mutable<ILogicalExpression> filterExpr : additionalFilteringExpressions) {
+                changed |= visitor.transform(filterExpr);
+            }
+        }
+        return changed;
+    }
+
     public boolean propagatesInput() {
         return propagateInput;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
index bb18014..9f73113 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
@@ -57,7 +57,7 @@
     }
 
     public DataSourceScanOperator(List<LogicalVariable> variables, IDataSource<?> dataSource,
-            Mutable<ILogicalExpression> selectCondition, long outputLimit, IProjectionInfo projectionInfo) {
+            Mutable<ILogicalExpression> selectCondition, long outputLimit, IProjectionInfo<?> projectionInfo) {
         super(variables, dataSource);
         projectVars = new ArrayList<>();
         this.selectCondition = selectCondition;
@@ -77,7 +77,13 @@
 
     @Override
     public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
-        return false;
+        boolean changed = selectCondition != null && visitor.transform(selectCondition);
+        if (additionalFilteringExpressions != null) {
+            for (Mutable<ILogicalExpression> filterExpr : additionalFilteringExpressions) {
+                changed |= visitor.transform(filterExpr);
+            }
+        }
+        return changed;
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/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/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/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/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-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/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-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-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/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/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/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
new file mode 100644
index 0000000..8ca1a82
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjector.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.common.projection;
+
+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;
+
+public interface ITupleProjector {
+    void project(ITupleReference tuple, DataOutput dos, ArrayTupleBuilder tb) throws IOException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
new file mode 100644
index 0000000..ff9ecf9
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/projection/ITupleProjectorFactory.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.common.projection;
+
+import java.io.Serializable;
+
+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;
+}