Merge branch 'stabilization-3b6982ce7f'

Change-Id: Ic5ceadf6cf4c2f5c2ac942470606823ec85118c5
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushGroupByThroughProduct.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushGroupByThroughProduct.java
index 48c4324..6d92b51 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushGroupByThroughProduct.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushGroupByThroughProduct.java
@@ -84,24 +84,27 @@
 
         Mutable<ILogicalOperator> opLeftRef = join.getInputs().get(0);
         ILogicalOperator opLeft = opLeftRef.getValue();
+        Mutable<ILogicalOperator> opRightRef = join.getInputs().get(1);
+        ILogicalOperator opRight = opRightRef.getValue();
         switch (canPushThrough(gby, opLeft, decorToPush, decorNotToPush)) {
             case REPEATED_DECORS: {
                 return false;
             }
             case TRUE: {
-                push(opRef, opRef2, 0, decorToPush, decorNotToPush, context);
-                return true;
+                if (OperatorPropertiesUtil.isCardinalityZeroOrOne(opRight)) {
+                    push(opRef, opRef2, 0, decorToPush, decorNotToPush, context);
+                    return true;
+                }
+                return false;
             }
             case FALSE: {
                 decorToPush.clear();
-                Mutable<ILogicalOperator> opRightRef = join.getInputs().get(1);
-                ILogicalOperator opRight = opRightRef.getValue();
-                if (canPushThrough(gby, opRight, decorToPush, decorNotToPush) == PushTestResult.TRUE) {
+                if (canPushThrough(gby, opRight, decorToPush, decorNotToPush) == PushTestResult.TRUE
+                        && OperatorPropertiesUtil.isCardinalityZeroOrOne(opLeft)) {
                     push(opRef, opRef2, 1, decorToPush, decorNotToPush, context);
                     return true;
-                } else {
-                    return false;
                 }
+                return false;
             }
             default: {
                 throw new IllegalStateException();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index b2fdb74..5bfdf1d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -126,9 +126,9 @@
     }
 
     protected void fillSubTreeIndexExprs(OptimizableOperatorSubTree subTree,
-            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs, IOptimizationContext context)
-            throws AlgebricksException {
-        fillSubTreeIndexExprs(subTree, analyzedAMs, context, false);
+            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs, IOptimizationContext context,
+            boolean isJoinLeftBranch) throws AlgebricksException {
+        fillSubTreeIndexExprs(subTree, analyzedAMs, context, isJoinLeftBranch, false);
     }
 
     /**
@@ -137,6 +137,7 @@
      * @param subTree
      * @param analyzedAMs
      * @param context
+     * @param isJoinLeftBranch
      * @param isArbitraryFormOfSubtree
      *            if the given subtree is in an arbitrary form that OptimizableSubTree class can't initialize, we try
      *            to fill the field type of each variable that is used in the optimizable function expressions.
@@ -146,7 +147,7 @@
      */
     protected void fillSubTreeIndexExprs(OptimizableOperatorSubTree subTree,
             Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs, IOptimizationContext context,
-            boolean isArbitraryFormOfSubtree) throws AlgebricksException {
+            boolean isJoinLeftBranch, boolean isArbitraryFormOfSubtree) throws AlgebricksException {
         Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
         // Check applicability of indexes by access method type.
         while (amIt.hasNext()) {
@@ -154,10 +155,13 @@
             AccessMethodAnalysisContext amCtx = entry.getValue();
             // For the current access method type, map variables to applicable
             // indexes.
-            if (!isArbitraryFormOfSubtree) {
-                fillAllIndexExprs(subTree, amCtx, context);
-            } else {
+            if (isArbitraryFormOfSubtree) {
                 fillVarFieldTypeForOptFuncExprs(subTree, amCtx, context);
+            } else {
+                if (isJoinLeftBranch) {
+                    fillVarFieldTypeForOptFuncExprs(subTree, amCtx, context);
+                }
+                fillAllIndexExprs(subTree, amCtx, context);
             }
         }
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
index 199f878..71baa5c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
@@ -330,12 +330,8 @@
             if (continueCheck && checkRightSubTreeMetadata) {
                 // Map variables to the applicable indexes and find the field name and type.
                 // Then find the applicable indexes for the variables used in the JOIN condition.
-                if (checkLeftSubTreeMetadata) {
-                    fillSubTreeIndexExprs(leftSubTree, analyzedAMs, context);
-                } else {
-                    fillSubTreeIndexExprs(leftSubTree, analyzedAMs, context, true);
-                }
-                fillSubTreeIndexExprs(rightSubTree, analyzedAMs, context);
+                fillSubTreeIndexExprs(leftSubTree, analyzedAMs, context, true, !checkLeftSubTreeMetadata);
+                fillSubTreeIndexExprs(rightSubTree, analyzedAMs, context, false);
 
                 // Prunes the access methods based on the function expression and access methods.
                 pruneIndexCandidates(analyzedAMs, context, typeEnvironment);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index 4199ece..7ed5994 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -395,7 +395,7 @@
             if (continueCheck) {
                 // Map variables to the applicable indexes and find the field name and type.
                 // Then find the applicable indexes for the variables used in the SELECT condition.
-                fillSubTreeIndexExprs(subTree, analyzedAMs, context);
+                fillSubTreeIndexExprs(subTree, analyzedAMs, context, false);
 
                 // Prune the access methods based on the function expression and access methods.
                 pruneIndexCandidates(analyzedAMs, context, typeEnvironment);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
index 1b7b7d2..4ac44b4 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
@@ -743,7 +743,7 @@
         gbyOp.getNestedPlans().add(nestedPlan);
 
         OperatorManipulationUtil.computeTypeEnvironmentBottomUp(gbyOp, context);
-        return op;
+        return gbyOp;
     }
 
     private ILogicalOperator visitMultiInputOperator(ILogicalOperator op) throws AlgebricksException {
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index 9131202..53f577f 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -525,8 +525,7 @@
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <version>1.10.19</version>
+      <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
index d6ebdad..2189c17 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
@@ -48,6 +48,7 @@
 import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.http.api.IChannelClosedHandler;
 import org.apache.hyracks.http.api.IServletRequest;
+import org.apache.hyracks.http.api.IServletResponse;
 import org.apache.hyracks.http.server.HttpServer;
 import org.apache.hyracks.http.server.InterruptOnCloseHandler;
 import org.apache.hyracks.ipc.exceptions.IPCException;
@@ -167,13 +168,13 @@
 
     @Override
     protected void handleExecuteStatementException(Throwable t, RequestExecutionState executionState,
-            QueryServiceRequestParameters param) {
+            QueryServiceRequestParameters param, IServletResponse response) {
         if (t instanceof TimeoutException // TODO(mblow): I don't think t can ever been an instance of TimeoutException
                 || ExceptionUtils.matchingCause(t, candidate -> candidate instanceof IPCException)) {
             GlobalConfig.ASTERIX_LOGGER.log(Level.WARN, t.toString(), t);
             executionState.setStatus(ResultStatus.FAILED, HttpResponseStatus.SERVICE_UNAVAILABLE);
         } else {
-            super.handleExecuteStatementException(t, executionState, param);
+            super.handleExecuteStatementException(t, executionState, param, response);
         }
     }
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
index a043050..abe9716 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
@@ -305,9 +305,9 @@
             }
             errorCount = 0;
         } catch (Exception | org.apache.asterix.lang.sqlpp.parser.TokenMgrError e) {
-            handleExecuteStatementException(e, executionState, param);
+            handleExecuteStatementException(e, executionState, param, response);
             response.setStatus(executionState.getHttpStatus());
-            requestFailed(e, responsePrinter);
+            requestFailed(e, responsePrinter, executionState);
         } finally {
             executionState.finish();
         }
@@ -419,7 +419,7 @@
     }
 
     protected boolean handleIFormattedException(IError error, IFormattedException ex,
-            RequestExecutionState executionState, QueryServiceRequestParameters param) {
+            RequestExecutionState executionState, QueryServiceRequestParameters param, IServletResponse response) {
         if (error instanceof ErrorCode) {
             switch ((ErrorCode) error) {
                 case INVALID_REQ_PARAM_VAL:
@@ -451,7 +451,7 @@
     }
 
     protected void handleExecuteStatementException(Throwable t, RequestExecutionState executionState,
-            QueryServiceRequestParameters param) {
+            QueryServiceRequestParameters param, IServletResponse response) {
         if (t instanceof org.apache.asterix.lang.sqlpp.parser.TokenMgrError || t instanceof AlgebricksException) {
             if (LOGGER.isDebugEnabled()) {
                 LOGGER.debug("handleException: {}: {}", t.getMessage(), LogRedactionUtil.statement(param.toString()),
@@ -465,8 +465,8 @@
         } else if (t instanceof IFormattedException) {
             IFormattedException formattedEx = (IFormattedException) t;
             Optional<IError> maybeError = formattedEx.getError();
-            if (maybeError.isPresent()
-                    && handleIFormattedException(maybeError.get(), (IFormattedException) t, executionState, param)) {
+            if (maybeError.isPresent() && handleIFormattedException(maybeError.get(), (IFormattedException) t,
+                    executionState, param, response)) {
                 return;
             }
         }
@@ -496,7 +496,8 @@
         sessionConfig.set(SessionConfig.FORMAT_CSV_HEADER, param.isCSVWithHeader());
     }
 
-    protected void requestFailed(Throwable throwable, ResponsePrinter responsePrinter) {
+    protected void requestFailed(Throwable throwable, ResponsePrinter responsePrinter,
+            RequestExecutionState executionState) {
         final ExecutionError executionError = ExecutionError.of(throwable);
         responsePrinter.addResultPrinter(new ErrorsPrinter(Collections.singletonList(executionError)));
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
index 29ee76d..30b98ec 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
@@ -139,6 +139,8 @@
         IStorageComponentProvider storageComponentProvider = ccAppCtx.getStorageComponentProvider();
         IStatementExecutorFactory statementExecutorFactory = ccApp.getStatementExecutorFactory();
         ExecuteStatementResponseMessage responseMsg = new ExecuteStatementResponseMessage(requestMessageId);
+        final IStatementExecutor.StatementProperties statementProperties = new IStatementExecutor.StatementProperties();
+        responseMsg.setStatementProperties(statementProperties);
         try {
             List<Warning> warnings = new ArrayList<>();
             IParser parser = compilationProvider.getParserFactory().createParser(statementsText);
@@ -160,8 +162,6 @@
                     compilationProvider, storageComponentProvider, new ResponsePrinter(sessionOutput));
             final IStatementExecutor.Stats stats = new IStatementExecutor.Stats();
             stats.setProfileType(profileType);
-            final IStatementExecutor.StatementProperties statementProperties =
-                    new IStatementExecutor.StatementProperties();
             Map<String, IAObject> stmtParams = RequestParameters.deserializeParameterValues(statementParameters);
             final IRequestParameters requestParameters =
                     new RequestParameters(requestReference, statementsText, null, resultProperties, stats,
@@ -174,7 +174,6 @@
             responseMsg.setResult(outWriter.toString());
             responseMsg.setMetadata(outMetadata);
             responseMsg.setStats(stats);
-            responseMsg.setStatementProperties(statementProperties);
             responseMsg.setExecutionPlans(translator.getExecutionPlans());
             responseMsg.setWarnings(warnings);
         } catch (AlgebricksException | HyracksException | org.apache.asterix.lang.sqlpp.parser.TokenMgrError pe) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 7396dcb..38225d7 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -1703,7 +1703,7 @@
             validateDatasetState(metadataProvider, ds, sourceLoc);
 
             ds.drop(metadataProvider, mdTxnCtx, jobsToExecute, bActiveTxn, progress, hcc, dropCorrespondingNodeGroup,
-                    sourceLoc, Collections.emptySet(), requestParameters.isForceDropDataset());
+                    sourceLoc, EnumSet.of(DropOption.IF_EXISTS), requestParameters.isForceDropDataset());
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
             return true;
@@ -1719,6 +1719,10 @@
                 try {
                     if (ds != null) {
                         jobsToExecute.clear();
+                        // start another txn for the compensating operations
+                        mdTxnCtx.setValue(MetadataManager.INSTANCE.beginTransaction());
+                        bActiveTxn.setValue(true);
+                        metadataProvider.setMetadataTxnContext(mdTxnCtx.getValue());
                         ds.drop(metadataProvider, mdTxnCtx, jobsToExecute, bActiveTxn, progress, hcc,
                                 dropCorrespondingNodeGroup, sourceLoc, EnumSet.of(DropOption.IF_EXISTS),
                                 requestParameters.isForceDropDataset());
@@ -3926,7 +3930,8 @@
         appCtx.getRequestTracker().track(clientRequest);
     }
 
-    protected void validateStatements(IRequestParameters requestParameters) throws CompilationException {
+    protected void validateStatements(IRequestParameters requestParameters)
+            throws CompilationException, HyracksDataException {
         validateStatements(statements, requestParameters.isMultiStatement(),
                 requestParameters.getStatementCategoryRestrictionMask());
     }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/IPollTask.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/IPollTask.java
index a1ed12b..8ba745b 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/IPollTask.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/IPollTask.java
@@ -19,7 +19,6 @@
 package org.apache.asterix.test.common;
 
 import java.io.File;
-import java.util.BitSet;
 import java.util.List;
 import java.util.Map;
 
@@ -32,8 +31,7 @@
 
     /**
      * Execute the poll task
-     *
-     * @param testCaseCtx
+     *  @param testCaseCtx
      * @param ctx
      * @param variableCtx
      * @param statement
@@ -44,11 +42,9 @@
      * @param expectedResultFileCtxs
      * @param testFile
      * @param actualPath
-     * @param expectedWarnings
      */
     void execute(TestCaseContext testCaseCtx, TestFileContext ctx, Map<String, Object> variableCtx, String statement,
             boolean isDmlRecoveryTest, ProcessBuilder pb, CompilationUnit cUnit, MutableInt queryCount,
-            List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath, BitSet expectedWarnings)
-            throws Exception;
+            List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath) throws Exception;
 
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
index 40946d3..f1bf63d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
@@ -1143,8 +1143,8 @@
 
     public void executeTestFile(TestCaseContext testCaseCtx, TestFileContext ctx, Map<String, Object> variableCtx,
             String statement, boolean isDmlRecoveryTest, ProcessBuilder pb, CompilationUnit cUnit,
-            MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath,
-            BitSet expectedWarnings) throws Exception {
+            MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath)
+            throws Exception {
         InputStream resultStream;
         File qbcFile;
         boolean failed = false;
@@ -1152,7 +1152,7 @@
         switch (ctx.getType()) {
             case "ddl":
                 ExtractedResult ddlExtractedResult = executeSqlppUpdateOrDdl(statement, OutputFormat.CLEAN_JSON, cUnit);
-                validateWarning(ddlExtractedResult, testCaseCtx, cUnit, testFile, expectedWarnings);
+                validateWarning(ddlExtractedResult, testCaseCtx, cUnit, testFile);
                 break;
             case "update":
                 // isDmlRecoveryTest: set IP address
@@ -1166,11 +1166,11 @@
             case "pollpost":
                 poll(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit, queryCount,
                         expectedResultFileCtxs, testFile, actualPath, ctx.getType().substring("poll".length()),
-                        expectedWarnings, plainExecutor);
+                        plainExecutor);
                 break;
             case "polldynamic":
                 polldynamic(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit, queryCount,
-                        expectedResultFileCtxs, testFile, actualPath, expectedWarnings);
+                        expectedResultFileCtxs, testFile, actualPath);
                 break;
             case "query":
             case "async":
@@ -1195,7 +1195,7 @@
                         variableCtx, ctx, expectedResultFile, actualResultFile, queryCount,
                         expectedResultFileCtxs.size(), cUnit.getParameter(), ComparisonEnum.TEXT);
 
-                validateWarning(extractedResult, testCaseCtx, cUnit, testFile, expectedWarnings);
+                validateWarning(extractedResult, testCaseCtx, cUnit, testFile);
                 break;
             case "store":
                 // This is a query that returns the expected output of a subsequent query
@@ -1610,18 +1610,18 @@
 
     private void polldynamic(TestCaseContext testCaseCtx, TestFileContext ctx, Map<String, Object> variableCtx,
             String statement, boolean isDmlRecoveryTest, ProcessBuilder pb, CompilationUnit cUnit,
-            MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath,
-            BitSet expectedWarnings) throws Exception {
+            MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath)
+            throws Exception {
         IExpectedResultPoller poller = getExpectedResultPoller(statement);
         final String key = getKey(statement);
         poll(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit, queryCount, expectedResultFileCtxs,
-                testFile, actualPath, "validate", expectedWarnings, new IPollTask() {
+                testFile, actualPath, "validate", new IPollTask() {
                     @Override
                     public void execute(TestCaseContext testCaseCtx, TestFileContext ctx,
                             Map<String, Object> variableCtx, String statement, boolean isDmlRecoveryTest,
                             ProcessBuilder pb, CompilationUnit cUnit, MutableInt queryCount,
-                            List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath,
-                            BitSet expectedWarnings) throws Exception {
+                            List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath)
+                            throws Exception {
                         File actualResultFile = new File(actualPath, testCaseCtx.getTestCase().getFilePath()
                                 + File.separatorChar + cUnit.getName() + '.' + ctx.getSeqNum() + ".polled.adm");
                         if (actualResultFile.exists() && !actualResultFile.delete()) {
@@ -1659,7 +1659,7 @@
     private void poll(TestCaseContext testCaseCtx, TestFileContext ctx, Map<String, Object> variableCtx,
             String statement, boolean isDmlRecoveryTest, ProcessBuilder pb, CompilationUnit cUnit,
             MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath,
-            String newType, BitSet expectedWarnings, IPollTask pollTask) throws Exception {
+            String newType, IPollTask pollTask) throws Exception {
         // polltimeoutsecs=nnn, polldelaysecs=nnn
         int timeoutSecs = getTimeoutSecs(statement);
         int retryDelaySecs = getRetryDelaySecs(statement);
@@ -1682,7 +1682,7 @@
                         try {
                             startSemaphore.release();
                             pollTask.execute(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit,
-                                    queryCount, expectedResultFileCtxs, testFile, actualPath, expectedWarnings);
+                                    queryCount, expectedResultFileCtxs, testFile, actualPath);
                         } finally {
                             endSemaphore.release();
                         }
@@ -2092,13 +2092,13 @@
     public void executeTest(String actualPath, TestCaseContext testCaseCtx, ProcessBuilder pb,
             boolean isDmlRecoveryTest, TestGroup failedGroup, TestGroup passedGroup) throws Exception {
         MutableInt queryCount = new MutableInt(0);
-        int numOfErrors = 0;
+        testCaseCtx.numOfErrors = 0;
         int numOfFiles = 0;
         List<CompilationUnit> cUnits = testCaseCtx.getTestCase().getCompilationUnit();
         for (CompilationUnit cUnit : cUnits) {
-            List<String> expectedErrors = cUnit.getExpectedError();
-            BitSet expectedWarnings = new BitSet(cUnit.getExpectedWarn().size());
-            expectedWarnings.set(0, cUnit.getExpectedWarn().size());
+            testCaseCtx.expectedErrors = cUnit.getExpectedError();
+            testCaseCtx.expectedWarnings = new BitSet(cUnit.getExpectedWarn().size());
+            testCaseCtx.expectedWarnings.set(0, cUnit.getExpectedWarn().size());
             LOGGER.info(
                     "Starting [TEST]: " + testCaseCtx.getTestCase().getFilePath() + "/" + cUnit.getName() + " ... ");
             Map<String, Object> variableCtx = new HashMap<>();
@@ -2120,7 +2120,7 @@
                     boolean loopCmd = testFile.getName().endsWith(".loop.cmd");
                     if (!testFile.getName().startsWith(DIAGNOSE)) {
                         executeTestFile(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit,
-                                queryCount, expectedResultFileCtxs, testFile, actualPath, expectedWarnings);
+                                queryCount, expectedResultFileCtxs, testFile, actualPath);
                     }
                     if (loopCmd) {
                         // this was a loop file and we have exited the loop; reset the loop iteration
@@ -2138,9 +2138,9 @@
                     }
                     loopIteration++;
                 } catch (Exception e) {
-                    numOfErrors++;
-                    boolean unexpected = isUnExpected(e, expectedErrors, numOfErrors, queryCount,
-                            testCaseCtx.isSourceLocationExpected(cUnit));
+                    testCaseCtx.numOfErrors++;
+                    boolean unexpected = isUnExpected(e, testCaseCtx.expectedErrors, testCaseCtx.numOfErrors,
+                            queryCount, testCaseCtx.isSourceLocationExpected(cUnit));
                     if (unexpected) {
                         LOGGER.error("testFile {} raised an unexpected exception", testFile, e);
                         if (failedGroup != null) {
@@ -2152,12 +2152,12 @@
                     }
                 }
                 if (numOfFiles == testFileCtxs.size()) {
-                    if (numOfErrors < cUnit.getExpectedError().size()) {
+                    if (testCaseCtx.numOfErrors < cUnit.getExpectedError().size()) {
                         LOGGER.error("Test {} failed to raise (an) expected exception(s)", cUnit.getName());
                         throw new Exception(
                                 "Test \"" + cUnit.getName() + "\" FAILED; expected exception was not thrown...");
                     }
-                    ensureWarnings(expectedWarnings, cUnit);
+                    ensureWarnings(testCaseCtx.expectedWarnings, cUnit);
                     LOGGER.info(
                             "[TEST]: " + testCaseCtx.getTestCase().getFilePath() + "/" + cUnit.getName() + " PASSED ");
                     if (passedGroup != null) {
@@ -2416,7 +2416,7 @@
                         final File file = ctx.getFile();
                         final String statement = readTestFile(file);
                         executeTestFile(testCaseCtx, ctx, variableCtx, statement, false, pb, cUnit, new MutableInt(-1),
-                                Collections.emptyList(), file, null, new BitSet());
+                                Collections.emptyList(), file, null);
                     }
                 }
             } catch (Exception diagnosticFailure) {
@@ -2794,11 +2794,11 @@
     }
 
     protected void validateWarning(ExtractedResult result, TestCaseContext testCaseCtx, CompilationUnit cUnit,
-            File testFile, BitSet expectedWarnings) throws Exception {
+            File testFile) throws Exception {
         if (testCaseCtx.getTestCase().isCheckWarnings()) {
             boolean expectedSourceLoc = testCaseCtx.isSourceLocationExpected(cUnit);
-            validateWarnings(result.getWarnings(), cUnit.getExpectedWarn(), expectedWarnings, expectedSourceLoc,
-                    testFile);
+            validateWarnings(result.getWarnings(), cUnit.getExpectedWarn(), testCaseCtx.expectedWarnings,
+                    expectedSourceLoc, testFile);
         }
     }
 
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/ExternalDatasetTestUtils.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/ExternalDatasetTestUtils.java
new file mode 100644
index 0000000..c064281
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/ExternalDatasetTestUtils.java
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.external_dataset;
+
+import static org.apache.asterix.test.external_dataset.aws.AwsS3ExternalDatasetTest.FIXED_DATA_CONTAINER;
+
+import java.io.BufferedWriter;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.commons.io.FilenameUtils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.Assert;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class ExternalDatasetTestUtils {
+
+    protected static final Logger LOGGER = LogManager.getLogger();
+
+    // Base directory paths for data files
+    private static String JSON_DATA_PATH;
+    private static String CSV_DATA_PATH;
+    private static String TSV_DATA_PATH;
+
+    // IMPORTANT: The following values must be used in the AWS S3 test case
+    // Region, container and definitions
+    public static final String JSON_DEFINITION = "json-data/reviews/";
+    public static final String CSV_DEFINITION = "csv-data/reviews/";
+    public static final String TSV_DEFINITION = "tsv-data/reviews/";
+    public static final String MIXED_DEFINITION = "mixed-data/reviews/";
+
+    // This is used for a test to generate over 1000 number of files
+    public static final String OVER_1000_OBJECTS_PATH = "over-1000-objects";
+    public static final int OVER_1000_OBJECTS_COUNT = 2999;
+
+    private static Uploader playgroundDataLoader;
+    private static Uploader fixedDataLoader;
+    private static Uploader mixedDataLoader;
+
+    protected TestCaseContext tcCtx;
+
+    public interface Uploader {
+        default void upload(String key, String content) {
+            upload(key, content, false, false);
+        }
+
+        void upload(String key, String content, boolean fromFile, boolean gzipped);
+    }
+
+    public ExternalDatasetTestUtils(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    public static void setDataPaths(String jsonDataPath, String csvDataPath, String tsvDataPath) {
+        JSON_DATA_PATH = jsonDataPath;
+        CSV_DATA_PATH = csvDataPath;
+        TSV_DATA_PATH = tsvDataPath;
+    }
+
+    public static void setUploaders(Uploader playgroundDataLoader, Uploader fixedDataLoader, Uploader mixedDataLoader) {
+        ExternalDatasetTestUtils.playgroundDataLoader = playgroundDataLoader;
+        ExternalDatasetTestUtils.fixedDataLoader = fixedDataLoader;
+        ExternalDatasetTestUtils.mixedDataLoader = mixedDataLoader;
+    }
+
+    /**
+     * Creates a bucket and fills it with some files for testing purpose.
+     */
+    public static void preparePlaygroundContainer() {
+        LOGGER.info("Adding JSON files to the bucket");
+        loadJsonFiles();
+        LOGGER.info("JSON Files added successfully");
+
+        LOGGER.info("Adding CSV files to the bucket");
+        loadCsvFiles();
+        LOGGER.info("CSV Files added successfully");
+
+        LOGGER.info("Adding TSV files to the bucket");
+        loadTsvFiles();
+        LOGGER.info("TSV Files added successfully");
+
+        LOGGER.info("Adding a big JSON file");
+        loadBigJson();
+        LOGGER.info("JSON file added successfully");
+
+        LOGGER.info("Loading " + OVER_1000_OBJECTS_COUNT + " into " + OVER_1000_OBJECTS_PATH);
+        loadLargeNumberOfFiles();
+        LOGGER.info("Added " + OVER_1000_OBJECTS_COUNT + " files into " + OVER_1000_OBJECTS_PATH + " successfully");
+
+        LOGGER.info("Files added successfully");
+    }
+
+    /**
+     * This bucket is being filled by fixed data, a test is counting all records in this bucket. If this bucket is
+     * changed, the test case will fail and its result will need to be updated each time
+     */
+    public static void prepareFixedDataContainer() {
+        LOGGER.info("Loading fixed data to " + FIXED_DATA_CONTAINER);
+
+        // Files data
+        String path = Paths.get(JSON_DATA_PATH, "single-line", "20-records.json").toString();
+        fixedDataLoader.upload("1.json", path, true, false);
+        fixedDataLoader.upload("2.json", path, true, false);
+        fixedDataLoader.upload("lvl1/3.json", path, true, false);
+        fixedDataLoader.upload("lvl1/34.json", path, true, false);
+        fixedDataLoader.upload("lvl1/lvl2/5.json", path, true, false);
+    }
+
+    public static void loadJsonFiles() {
+        String dataBasePath = JSON_DATA_PATH;
+        String definition = JSON_DEFINITION;
+
+        // Normal format
+        String definitionSegment = "json";
+        loadData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
+        loadData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
+        loadData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
+        loadData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
+                false);
+
+        definitionSegment = "json-array-of-objects";
+        loadData(dataBasePath, "single-line", "array_of_objects.json", "json-data/", definitionSegment, false, false);
+
+        // gz compressed format
+        definitionSegment = "gz";
+        loadGzData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
+        loadGzData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
+        loadGzData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
+        loadGzData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
+                false);
+
+        // Mixed normal and gz compressed format
+        definitionSegment = "mixed";
+        loadData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
+        loadData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
+        loadData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
+        loadData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
+                false);
+        loadGzData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
+        loadGzData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
+        loadGzData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
+        loadGzData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
+                false);
+    }
+
+    private static void loadCsvFiles() {
+        String dataBasePath = CSV_DATA_PATH;
+        String definition = CSV_DEFINITION;
+
+        // Normal format
+        String definitionSegment = "csv";
+        loadData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
+        loadData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
+
+        // gz compressed format
+        definitionSegment = "gz";
+        loadGzData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
+        loadGzData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
+
+        // Mixed normal and gz compressed format
+        definitionSegment = "mixed";
+        loadData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
+        loadData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
+        loadGzData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
+        loadGzData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
+    }
+
+    private static void loadTsvFiles() {
+        String dataBasePath = TSV_DATA_PATH;
+        String definition = TSV_DEFINITION;
+
+        // Normal format
+        String definitionSegment = "tsv";
+        loadData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
+        loadData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
+
+        // gz compressed format
+        definitionSegment = "gz";
+        loadGzData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
+        loadGzData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
+
+        // Mixed normal and gz compressed format
+        definitionSegment = "mixed";
+        loadData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
+        loadData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
+        loadGzData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
+        loadGzData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
+    }
+
+    private static void loadData(String fileBasePath, String filePathSegment, String filename, String definition,
+            String definitionSegment, boolean removeExtension) {
+        loadData(fileBasePath, filePathSegment, filename, definition, definitionSegment, removeExtension, true);
+    }
+
+    private static void loadData(String fileBasePath, String filePathSegment, String filename, String definition,
+            String definitionSegment, boolean removeExtension, boolean copyToSubLevels) {
+        // Files data
+        Path filePath = Paths.get(fileBasePath, filePathSegment, filename);
+
+        // Keep or remove the file extension
+        Assert.assertFalse("Files with no extension are not supported yet for external datasets", removeExtension);
+        String finalFileName;
+        if (removeExtension) {
+            finalFileName = FilenameUtils.removeExtension(filename);
+        } else {
+            finalFileName = filename;
+        }
+
+        // Files base definition
+        filePathSegment = filePathSegment.isEmpty() ? "" : filePathSegment + "/";
+        definitionSegment = definitionSegment.isEmpty() ? "" : definitionSegment + "/";
+        String basePath = definition + filePathSegment + definitionSegment;
+
+        // Load the data
+        String path = filePath.toString();
+        playgroundDataLoader.upload(basePath + finalFileName, path, true, false);
+        if (copyToSubLevels) {
+            playgroundDataLoader.upload(basePath + "level1a/" + finalFileName, path, true, false);
+            playgroundDataLoader.upload(basePath + "level1b/" + finalFileName, path, true, false);
+            playgroundDataLoader.upload(basePath + "level1a/level2a/" + finalFileName, path, true, false);
+            playgroundDataLoader.upload(basePath + "level1a/level2b/" + finalFileName, path, true, false);
+        }
+    }
+
+    private static void loadGzData(String fileBasePath, String filePathSegment, String filename, String definition,
+            String definitionSegment, boolean removeExtension) {
+        // Keep or remove the file extension
+        Assert.assertFalse("Files with no extension are not supported yet for external datasets", removeExtension);
+        String finalFileName;
+        if (removeExtension) {
+            finalFileName = FilenameUtils.removeExtension(filename);
+        } else {
+            finalFileName = filename;
+        }
+        finalFileName += ".gz";
+
+        // Files base definition
+        filePathSegment = filePathSegment.isEmpty() ? "" : filePathSegment + "/";
+        definitionSegment = definitionSegment.isEmpty() ? "" : definitionSegment + "/";
+        String basePath = definition + filePathSegment + definitionSegment;
+
+        // Load the data
+        String path = Paths.get(fileBasePath, filePathSegment, filename).toString();
+        playgroundDataLoader.upload(basePath + finalFileName, path, true, true);
+        playgroundDataLoader.upload(basePath + "level1a/" + finalFileName, path, true, true);
+        playgroundDataLoader.upload(basePath + "level1b/" + finalFileName, path, true, true);
+        playgroundDataLoader.upload(basePath + "level1a/level2a/" + finalFileName, path, true, true);
+        playgroundDataLoader.upload(basePath + "level1a/level2b/" + finalFileName, path, true, true);
+    }
+
+    private static void loadBigJson() {
+        String fileName = "big_record.json";
+        int bufferSize = 4 * 1024 * 1024;
+        int maxSize = bufferSize * 9;
+        Path filePath = Paths.get("target", "rttest", "tmp", fileName);
+        try {
+            if (Files.notExists(filePath)) {
+                Files.createDirectories(filePath.getParent());
+                Files.createFile(filePath);
+            }
+        } catch (IOException ex) {
+            throw new IllegalStateException("File " + fileName + " not found");
+        }
+
+        try (FileWriter writer = new FileWriter(filePath.toFile(), false);
+                BufferedWriter bw = new BufferedWriter(writer, bufferSize)) {
+            bw.append("{ \"large_field\": \"");
+            for (int i = 0; i < maxSize; i++) {
+                bw.append('A');
+            }
+            bw.append("\" }");
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+        String key = "big-json/" + fileName;
+        playgroundDataLoader.upload(key, filePath.toString(), true, false);
+    }
+
+    /**
+     * Generates over 1000 objects and upload them to S3 mock server, 1 record per object
+     */
+    private static void loadLargeNumberOfFiles() {
+        for (int i = 0; i < OVER_1000_OBJECTS_COUNT; i++) {
+            playgroundDataLoader.upload(OVER_1000_OBJECTS_PATH + "/" + i + ".json", "{\"id\":" + i + "}");
+        }
+    }
+
+    /**
+     * Loads a combination of different file formats in the same path
+     */
+    public static void prepareMixedDataContainer() {
+        // JSON
+        mixedDataLoader.upload(MIXED_DEFINITION + "json/extension/" + "hello-world-2018.json", "{\"id\":" + 1 + "}");
+        mixedDataLoader.upload(MIXED_DEFINITION + "json/extension/" + "hello-world-2019.json", "{\"id\":" + 2 + "}");
+        mixedDataLoader.upload(MIXED_DEFINITION + "json/extension/" + "hello-world-2020.json", "{\"id\":" + 3 + "}");
+        mixedDataLoader.upload(MIXED_DEFINITION + "json/EXTENSION/" + "goodbye-world-2018.json", "{\"id\":" + 4 + "}");
+        mixedDataLoader.upload(MIXED_DEFINITION + "json/EXTENSION/" + "goodbye-world-2019.json", "{\"id\":" + 5 + "}");
+        mixedDataLoader.upload(MIXED_DEFINITION + "json/EXTENSION/" + "goodbye-world-2020.json", "{\"id\":" + 6 + "}");
+
+        // CSV
+        mixedDataLoader.upload(MIXED_DEFINITION + "csv/extension/" + "hello-world-2018.csv", "7,\"good\"");
+        mixedDataLoader.upload(MIXED_DEFINITION + "csv/extension/" + "hello-world-2019.csv", "8,\"good\"");
+        mixedDataLoader.upload(MIXED_DEFINITION + "csv/extension/" + "hello-world-2020.csv", "9,\"good\"");
+        mixedDataLoader.upload(MIXED_DEFINITION + "csv/EXTENSION/" + "goodbye-world-2018.csv", "10,\"good\"");
+        mixedDataLoader.upload(MIXED_DEFINITION + "csv/EXTENSION/" + "goodbye-world-2019.csv", "11,\"good\"");
+        mixedDataLoader.upload(MIXED_DEFINITION + "csv/EXTENSION/" + "goodbye-world-2020.csv", "12,\"good\"");
+
+        // TSV
+        mixedDataLoader.upload(MIXED_DEFINITION + "tsv/extension/" + "hello-world-2018.tsv", "13\t\"good\"");
+        mixedDataLoader.upload(MIXED_DEFINITION + "tsv/extension/" + "hello-world-2019.tsv", "14\t\"good\"");
+        mixedDataLoader.upload(MIXED_DEFINITION + "tsv/extension/" + "hello-world-2020.tsv", "15\t\"good\"");
+        mixedDataLoader.upload(MIXED_DEFINITION + "tsv/EXTENSION/" + "goodbye-world-2018.tsv", "16\t\"good\"");
+        mixedDataLoader.upload(MIXED_DEFINITION + "tsv/EXTENSION/" + "goodbye-world-2019.tsv", "17\t\"good\"");
+        mixedDataLoader.upload(MIXED_DEFINITION + "tsv/EXTENSION/" + "goodbye-world-2020.tsv", "18\t\"good\"");
+
+        // JSON no extension
+        mixedDataLoader.upload(MIXED_DEFINITION + "json/no-extension/" + "hello-world-2018", "{\"id\":" + 1 + "}");
+        mixedDataLoader.upload(MIXED_DEFINITION + "json/no-extension/" + "hello-world-2019", "{\"id\":" + 2 + "}");
+        mixedDataLoader.upload(MIXED_DEFINITION + "json/no-extension/" + "hello-world-2020", "{\"id\":" + 3 + "}");
+        mixedDataLoader.upload(MIXED_DEFINITION + "json/NO-EXTENSION/" + "goodbye-world-2018", "{\"id\":" + 4 + "}");
+        mixedDataLoader.upload(MIXED_DEFINITION + "json/NO-EXTENSION/" + "goodbye-world-2019", "{\"id\":" + 5 + "}");
+        mixedDataLoader.upload(MIXED_DEFINITION + "json/NO-EXTENSION/" + "goodbye-world-2020", "{\"id\":" + 6 + "}");
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetTest.java
index 387f7f6..c3c94f6 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetTest.java
@@ -18,17 +18,18 @@
  */
 package org.apache.asterix.test.external_dataset.aws;
 
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.setDataPaths;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.setUploaders;
 import static org.apache.hyracks.util.file.FileUtil.joinPath;
 
 import java.io.ByteArrayOutputStream;
 import java.io.File;
+import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.nio.file.Files;
-import java.nio.file.Path;
 import java.nio.file.Paths;
-import java.util.BitSet;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -39,6 +40,7 @@
 
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils;
 import org.apache.asterix.test.runtime.ExecutionTestUtil;
 import org.apache.asterix.test.runtime.LangExecutionUtil;
 import org.apache.asterix.testframework.context.TestCaseContext;
@@ -50,7 +52,6 @@
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.FixMethodOrder;
 import org.junit.Test;
@@ -91,7 +92,6 @@
     private static final String JSON_DATA_PATH = joinPath("data", "json");
     private static final String CSV_DATA_PATH = joinPath("data", "csv");
     private static final String TSV_DATA_PATH = joinPath("data", "tsv");
-    private static final String MIXED_DATA_PATH = joinPath("data", "mixed");
 
     // Service endpoint
     private static final int MOCK_SERVER_PORT = 8001;
@@ -99,31 +99,27 @@
 
     // Region, bucket and definitions
     private static final String MOCK_SERVER_REGION = "us-west-2";
-    private static final String MOCK_SERVER_BUCKET = "playground";
-    private static final String FIXED_DATA_BUCKET = "fixed-data"; // Do not use, has fixed data
-    private static final String INCLUDE_EXCLUDE_BUCKET = "include-exclude"; // include & exclude bucket
-    private static final String JSON_DEFINITION = "json-data/reviews/"; // data resides here
-    private static final String CSV_DEFINITION = "csv-data/reviews/"; // data resides here
-    private static final String TSV_DEFINITION = "tsv-data/reviews/"; // data resides here
-
-    // This is used for a test to generate over 1000 number of files
-    private static final String OVER_1000_OBJECTS_PATH = "over-1000-objects";
-    private static final int OVER_1000_OBJECTS_COUNT = 2999;
 
     private static final Set<String> fileNames = new HashSet<>();
     private static final CreateBucketRequest.Builder CREATE_BUCKET_BUILDER = CreateBucketRequest.builder();
     private static final DeleteBucketRequest.Builder DELETE_BUCKET_BUILDER = DeleteBucketRequest.builder();
     private static final PutObjectRequest.Builder PUT_OBJECT_BUILDER = PutObjectRequest.builder();
 
-    // IMPORTANT: The following values must be used in the AWS S3 test case
     private static S3Mock s3MockServer;
     private static S3Client client;
-    private static final PutObjectRequest.Builder builder = PutObjectRequest.builder().bucket(MOCK_SERVER_BUCKET);
-    private static final PutObjectRequest.Builder includeExcludeBuilder =
-            PutObjectRequest.builder().bucket(INCLUDE_EXCLUDE_BUCKET);
 
     protected TestCaseContext tcCtx;
 
+    public static final String PLAYGROUND_CONTAINER = "playground";
+    public static final String FIXED_DATA_CONTAINER = "fixed-data"; // Do not use, has fixed data
+    public static final String INCLUDE_EXCLUDE_CONTAINER = "include-exclude";
+    public static final PutObjectRequest.Builder playgroundBuilder =
+            PutObjectRequest.builder().bucket(PLAYGROUND_CONTAINER);
+    public static final PutObjectRequest.Builder fixedDataBuilder =
+            PutObjectRequest.builder().bucket(FIXED_DATA_CONTAINER);
+    public static final PutObjectRequest.Builder includeExcludeBuilder =
+            PutObjectRequest.builder().bucket(INCLUDE_EXCLUDE_CONTAINER);
+
     public AwsS3ExternalDatasetTest(TestCaseContext tcCtx) {
         this.tcCtx = tcCtx;
     }
@@ -156,9 +152,9 @@
         SUITE_TESTS = "testsuite_external_dataset_s3.xml";
         ONLY_TESTS = "only_external_dataset.xml";
         TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
-        PREPARE_BUCKET = AwsS3ExternalDatasetTest::prepareS3Bucket;
-        PREPARE_FIXED_DATA_BUCKET = AwsS3ExternalDatasetTest::prepareFixedDataBucket;
-        PREPARE_MIXED_DATA_BUCKET = AwsS3ExternalDatasetTest::prepareMixedDataBucket;
+        PREPARE_BUCKET = ExternalDatasetTestUtils::preparePlaygroundContainer;
+        PREPARE_FIXED_DATA_BUCKET = ExternalDatasetTestUtils::prepareFixedDataContainer;
+        PREPARE_MIXED_DATA_BUCKET = ExternalDatasetTestUtils::prepareMixedDataContainer;
         return LangExecutionUtil.tests(ONLY_TESTS, SUITE_TESTS);
     }
 
@@ -197,320 +193,64 @@
         builder.region(Region.of(MOCK_SERVER_REGION)).credentialsProvider(AnonymousCredentialsProvider.create())
                 .endpointOverride(endpoint);
         client = builder.build();
+        client.createBucket(CreateBucketRequest.builder().bucket(PLAYGROUND_CONTAINER).build());
+        client.createBucket(CreateBucketRequest.builder().bucket(FIXED_DATA_CONTAINER).build());
+        client.createBucket(CreateBucketRequest.builder().bucket(INCLUDE_EXCLUDE_CONTAINER).build());
         LOGGER.info("Client created successfully");
 
         // Create the bucket and upload some json files
+        setDataPaths(JSON_DATA_PATH, CSV_DATA_PATH, TSV_DATA_PATH);
+        setUploaders(AwsS3ExternalDatasetTest::loadPlaygroundData, AwsS3ExternalDatasetTest::loadFixedData,
+                AwsS3ExternalDatasetTest::loadMixedData);
         PREPARE_BUCKET.run();
         PREPARE_FIXED_DATA_BUCKET.run();
         PREPARE_MIXED_DATA_BUCKET.run();
     }
 
-    /**
-     * Creates a bucket and fills it with some files for testing purpose.
-     */
-    private static void prepareS3Bucket() {
-        LOGGER.info("creating bucket " + MOCK_SERVER_BUCKET);
-        client.createBucket(CreateBucketRequest.builder().bucket(MOCK_SERVER_BUCKET).build());
-        LOGGER.info("bucket created successfully");
-
-        LOGGER.info("Adding JSON files to the bucket");
-        loadJsonFiles();
-        LOGGER.info("JSON Files added successfully");
-
-        LOGGER.info("Adding CSV files to the bucket");
-        loadCsvFiles();
-        LOGGER.info("CSV Files added successfully");
-
-        LOGGER.info("Adding TSV files to the bucket");
-        loadTsvFiles();
-        LOGGER.info("TSV Files added successfully");
-
-        LOGGER.info("Loading " + OVER_1000_OBJECTS_COUNT + " into " + OVER_1000_OBJECTS_PATH);
-        loadLargeNumberOfFiles();
-        LOGGER.info("Added " + OVER_1000_OBJECTS_COUNT + " files into " + OVER_1000_OBJECTS_PATH + " successfully");
+    private static void loadPlaygroundData(String key, String content, boolean fromFile, boolean gzipped) {
+        client.putObject(playgroundBuilder.key(key).build(), getRequestBody(content, fromFile, gzipped));
     }
 
-    /**
-     * This bucket is being filled by fixed data, a test is counting all records in this bucket. If this bucket is
-     * changed, the test case will fail and its result will need to be updated each time
-     */
-    private static void prepareFixedDataBucket() {
-        LOGGER.info("creating bucket " + FIXED_DATA_BUCKET);
-        client.createBucket(CreateBucketRequest.builder().bucket(FIXED_DATA_BUCKET).build());
-        LOGGER.info("bucket " + FIXED_DATA_BUCKET + " created successfully");
-
-        LOGGER.info("Loading fixed data to " + FIXED_DATA_BUCKET);
-
-        // Files data
-        RequestBody requestBody = RequestBody.fromFile(Paths.get(JSON_DATA_PATH, "single-line", "20-records.json"));
-        client.putObject(builder.bucket(FIXED_DATA_BUCKET).key("1.json").build(), requestBody);
-        client.putObject(builder.bucket(FIXED_DATA_BUCKET).key("2.json").build(), requestBody);
-        client.putObject(builder.bucket(FIXED_DATA_BUCKET).key("lvl1/3.json").build(), requestBody);
-        client.putObject(builder.bucket(FIXED_DATA_BUCKET).key("lvl1/4.json").build(), requestBody);
-        client.putObject(builder.bucket(FIXED_DATA_BUCKET).key("lvl1/lvl2/5.json").build(), requestBody);
+    private static void loadFixedData(String key, String content, boolean fromFile, boolean gzipped) {
+        client.putObject(fixedDataBuilder.key(key).build(), getRequestBody(content, fromFile, gzipped));
     }
 
-    private static void loadJsonFiles() {
-        String dataBasePath = JSON_DATA_PATH;
-        String definition = JSON_DEFINITION;
-
-        // Normal format
-        String definitionSegment = "json";
-        loadData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
-        loadData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
-        loadData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
-        loadData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
-                false);
-
-        definitionSegment = "json-array-of-objects";
-        loadData(dataBasePath, "single-line", "array_of_objects.json", "json-data/", definitionSegment, false, false);
-
-        // gz compressed format
-        definitionSegment = "gz";
-        loadGzData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
-        loadGzData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
-        loadGzData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
-        loadGzData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
-                false);
-
-        // Mixed normal and gz compressed format
-        definitionSegment = "mixed";
-        loadData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
-        loadData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
-        loadData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
-        loadData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
-                false);
-        loadGzData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
-        loadGzData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
-        loadGzData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
-        loadGzData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
-                false);
+    private static void loadMixedData(String key, String content, boolean fromFile, boolean gzipped) {
+        client.putObject(includeExcludeBuilder.key(key).build(), getRequestBody(content, fromFile, gzipped));
     }
 
-    private static void loadCsvFiles() {
-        String dataBasePath = CSV_DATA_PATH;
-        String definition = CSV_DEFINITION;
+    private static RequestBody getRequestBody(String content, boolean fromFile, boolean gzipped) {
+        RequestBody body;
 
-        // Normal format
-        String definitionSegment = "csv";
-        loadData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
-        loadData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
-
-        // gz compressed format
-        definitionSegment = "gz";
-        loadGzData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
-        loadGzData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
-
-        // Mixed normal and gz compressed format
-        definitionSegment = "mixed";
-        loadData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
-        loadData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
-        loadGzData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
-        loadGzData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
-    }
-
-    private static void loadTsvFiles() {
-        String dataBasePath = TSV_DATA_PATH;
-        String definition = TSV_DEFINITION;
-
-        // Normal format
-        String definitionSegment = "tsv";
-        loadData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
-        loadData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
-
-        // gz compressed format
-        definitionSegment = "gz";
-        loadGzData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
-        loadGzData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
-
-        // Mixed normal and gz compressed format
-        definitionSegment = "mixed";
-        loadData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
-        loadData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
-        loadGzData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
-        loadGzData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
-    }
-
-    private static void loadData(String fileBasePath, String filePathSegment, String filename, String definition,
-            String definitionSegment, boolean removeExtension) {
-        loadData(fileBasePath, filePathSegment, filename, definition, definitionSegment, removeExtension, true);
-    }
-
-    private static void loadData(String fileBasePath, String filePathSegment, String filename, String definition,
-            String definitionSegment, boolean removeExtension, boolean copyToSubLevels) {
-        // Files data
-        Path filePath = Paths.get(fileBasePath, filePathSegment, filename);
-        RequestBody requestBody = RequestBody.fromFile(filePath);
-
-        // Keep or remove the file extension
-        Assert.assertFalse("Files with no extension are not supported yet for external datasets", removeExtension);
-        String finalFileName;
-        if (removeExtension) {
-            finalFileName = FilenameUtils.removeExtension(filename);
+        // Content is string
+        if (!fromFile) {
+            body = RequestBody.fromString(content);
         } else {
-            finalFileName = filename;
-        }
-
-        // Files base definition
-        filePathSegment = filePathSegment.isEmpty() ? "" : filePathSegment + "/";
-        definitionSegment = definitionSegment.isEmpty() ? "" : definitionSegment + "/";
-        String basePath = definition + filePathSegment + definitionSegment;
-
-        // Load the data
-        client.putObject(builder.key(basePath + finalFileName).build(), requestBody);
-        if (copyToSubLevels) {
-            client.putObject(builder.key(basePath + "level1a/" + finalFileName).build(), requestBody);
-            client.putObject(builder.key(basePath + "level1b/" + finalFileName).build(), requestBody);
-            client.putObject(builder.key(basePath + "level1a/level2a/" + finalFileName).build(), requestBody);
-            client.putObject(builder.key(basePath + "level1a/level2b/" + finalFileName).build(), requestBody);
-        }
-    }
-
-    private static void loadGzData(String fileBasePath, String filePathSegment, String filename, String definition,
-            String definitionSegment, boolean removeExtension) {
-        try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-                GZIPOutputStream gzipOutputStream = new GZIPOutputStream(byteArrayOutputStream)) {
-
-            // Files data
-            Path filePath = Paths.get(fileBasePath, filePathSegment, filename);
-
-            // Get the compressed data
-            gzipOutputStream.write(Files.readAllBytes(filePath));
-            gzipOutputStream.close(); // Need to close or data will be invalid
-            byte[] gzipBytes = byteArrayOutputStream.toByteArray();
-            RequestBody requestBody = RequestBody.fromBytes(gzipBytes);
-
-            // Keep or remove the file extension
-            Assert.assertFalse("Files with no extension are not supported yet for external datasets", removeExtension);
-            String finalFileName;
-            if (removeExtension) {
-                finalFileName = FilenameUtils.removeExtension(filename);
+            // Content is a file path
+            if (!gzipped) {
+                body = RequestBody.fromFile(Paths.get(content));
             } else {
-                finalFileName = filename;
+                try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+                        GZIPOutputStream gzipOutputStream = new GZIPOutputStream(byteArrayOutputStream)) {
+                    gzipOutputStream.write(Files.readAllBytes(Paths.get(content)));
+                    gzipOutputStream.close(); // Need to close or data will be invalid
+                    byte[] gzipBytes = byteArrayOutputStream.toByteArray();
+                    body = RequestBody.fromBytes(gzipBytes);
+                } catch (IOException ex) {
+                    throw new IllegalArgumentException(ex.toString());
+                }
             }
-            finalFileName += ".gz";
-
-            // Files base definition
-            filePathSegment = filePathSegment.isEmpty() ? "" : filePathSegment + "/";
-            definitionSegment = definitionSegment.isEmpty() ? "" : definitionSegment + "/";
-            String basePath = definition + filePathSegment + definitionSegment;
-
-            // Load the data
-            client.putObject(builder.key(basePath + finalFileName).build(), requestBody);
-            client.putObject(builder.key(basePath + "level1a/" + finalFileName).build(), requestBody);
-            client.putObject(builder.key(basePath + "level1b/" + finalFileName).build(), requestBody);
-            client.putObject(builder.key(basePath + "level1a/level2a/" + finalFileName).build(), requestBody);
-            client.putObject(builder.key(basePath + "level1a/level2b/" + finalFileName).build(), requestBody);
-        } catch (Exception ex) {
-            LOGGER.error(ex.getMessage());
         }
-    }
 
-    /**
-     * Generates over 1000 objects and upload them to S3 mock server, 1 record per object
-     */
-    private static void loadLargeNumberOfFiles() {
-        for (int i = 0; i < OVER_1000_OBJECTS_COUNT; i++) {
-            RequestBody body = RequestBody.fromString("{\"id\":" + i + "}");
-            client.putObject(builder.key(OVER_1000_OBJECTS_PATH + "/" + i + ".json").build(), body);
-        }
-    }
-
-    /**
-     * Loads a combination of different file formats in the same path
-     */
-    private static void prepareMixedDataBucket() {
-        LOGGER.info("creating bucket " + INCLUDE_EXCLUDE_BUCKET);
-        client.createBucket(CreateBucketRequest.builder().bucket(INCLUDE_EXCLUDE_BUCKET).build());
-        LOGGER.info("bucket " + INCLUDE_EXCLUDE_BUCKET + " created successfully");
-
-        // JSON
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/extension/" + "hello-world-2018.json").build(),
-                RequestBody.fromString("{\"id\":" + 1 + "}"));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/extension/" + "hello-world-2019.json").build(),
-                RequestBody.fromString("{\"id\":" + 2 + "}"));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/extension/" + "hello-world-2020.json").build(),
-                RequestBody.fromString("{\"id\":" + 3 + "}"));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/EXTENSION/" + "goodbye-world-2018.json").build(),
-                RequestBody.fromString("{\"id\":" + 4 + "}"));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/EXTENSION/" + "goodbye-world-2019.json").build(),
-                RequestBody.fromString("{\"id\":" + 5 + "}"));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/EXTENSION/" + "goodbye-world-2020.json").build(),
-                RequestBody.fromString("{\"id\":" + 6 + "}"));
-
-        // CSV
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/extension/" + "hello-world-2018.csv").build(),
-                RequestBody.fromString("7,\"good\""));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/extension/" + "hello-world-2019.csv").build(),
-                RequestBody.fromString("8,\"good\""));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/extension/" + "hello-world-2020.csv").build(),
-                RequestBody.fromString("{9,\"good\""));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/EXTENSION/" + "goodbye-world-2018.csv").build(),
-                RequestBody.fromString("10,\"good\""));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/EXTENSION/" + "goodbye-world-2019.csv").build(),
-                RequestBody.fromString("11,\"good\""));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/EXTENSION/" + "goodbye-world-2020.csv").build(),
-                RequestBody.fromString("12,\"good\""));
-
-        // TSV
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/extension/" + "hello-world-2018.tsv").build(),
-                RequestBody.fromString("13\t\"good\""));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/extension/" + "hello-world-2019.tsv").build(),
-                RequestBody.fromString("14\t\"good\""));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/extension/" + "hello-world-2020.tsv").build(),
-                RequestBody.fromString("15\t\"good\""));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/EXTENSION/" + "goodbye-world-2018.tsv").build(),
-                RequestBody.fromString("16\t\"good\""));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/EXTENSION/" + "goodbye-world-2019.tsv").build(),
-                RequestBody.fromString("17\t\"good\""));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/EXTENSION/" + "goodbye-world-2020.tsv").build(),
-                RequestBody.fromString("18\t\"good\""));
-
-        // JSON no extension
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/no-extension/" + "hello-world-2018").build(),
-                RequestBody.fromString("{\"id\":" + 1 + "}"));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/no-extension/" + "hello-world-2019").build(),
-                RequestBody.fromString("{\"id\":" + 2 + "}"));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/no-extension/" + "hello-world-2020").build(),
-                RequestBody.fromString("{\"id\":" + 3 + "}"));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/NO-EXTENSION/" + "goodbye-world-2018").build(),
-                RequestBody.fromString("{\"id\":" + 4 + "}"));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/NO-EXTENSION/" + "goodbye-world-2019").build(),
-                RequestBody.fromString("{\"id\":" + 5 + "}"));
-        client.putObject(
-                includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/NO-EXTENSION/" + "goodbye-world-2020").build(),
-                RequestBody.fromString("{\"id\":" + 6 + "}"));
+        return body;
     }
 
     static class AwsTestExecutor extends TestExecutor {
 
         public void executeTestFile(TestCaseContext testCaseCtx, TestFileContext ctx, Map<String, Object> variableCtx,
                 String statement, boolean isDmlRecoveryTest, ProcessBuilder pb, TestCase.CompilationUnit cUnit,
-                MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath,
-                BitSet expectedWarnings) throws Exception {
+                MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath)
+                throws Exception {
             String[] lines;
             switch (ctx.getType()) {
                 case "container":
@@ -526,7 +266,7 @@
                     break;
                 default:
                     super.executeTestFile(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit,
-                            queryCount, expectedResultFileCtxs, testFile, actualPath, expectedWarnings);
+                            queryCount, expectedResultFileCtxs, testFile, actualPath);
             }
         }
     }
@@ -588,4 +328,4 @@
         }
         LOGGER.info("Done creating bucket with data");
     }
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/microsoft/AzureBlobStorageExternalDatasetTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/microsoft/AzureBlobStorageExternalDatasetTest.java
index 27a46c1..2d41686 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/microsoft/AzureBlobStorageExternalDatasetTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/microsoft/AzureBlobStorageExternalDatasetTest.java
@@ -31,7 +31,6 @@
 import java.nio.file.Paths;
 import java.time.OffsetDateTime;
 import java.time.temporal.ChronoUnit;
-import java.util.BitSet;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -67,11 +66,13 @@
 import com.azure.storage.blob.BlobContainerClient;
 import com.azure.storage.blob.BlobServiceClient;
 import com.azure.storage.blob.BlobServiceClientBuilder;
+import com.azure.storage.blob.models.PublicAccessType;
 import com.azure.storage.common.sas.AccountSasPermission;
 import com.azure.storage.common.sas.AccountSasResourceType;
 import com.azure.storage.common.sas.AccountSasService;
 import com.azure.storage.common.sas.AccountSasSignatureValues;
 
+// TODO(Hussain): Need to run the test manually to ensure new tests (anonymous access) are working fine
 @Ignore
 @RunWith(Parameterized.class)
 @FixMethodOrder(MethodSorters.NAME_ASCENDING)
@@ -101,6 +102,7 @@
     private static final String PLAYGROUND_CONTAINER = "playground";
     private static final String FIXED_DATA_CONTAINER = "fixed-data"; // Do not use, has fixed data
     private static final String INCLUDE_EXCLUDE_CONTAINER = "include-exclude";
+    private static final String PUBLIC_ACCESS_CONTAINER = "public-access-container"; // requires no authentication
     private static final String JSON_DEFINITION = "json-data/reviews/";
     private static final String CSV_DEFINITION = "csv-data/reviews/";
     private static final String TSV_DEFINITION = "tsv-data/reviews/";
@@ -117,6 +119,7 @@
             + "BlobEndpoint=" + BLOB_SERVICE_ENDPOINT + "/devstoreaccount1;";
     private static BlobServiceClient blobServiceClient;
     private static BlobContainerClient playgroundContainer;
+    private static BlobContainerClient publicAccessContainer;
 
     protected TestCaseContext tcCtx;
 
@@ -196,6 +199,8 @@
 
         LOGGER.info("creating container " + PLAYGROUND_CONTAINER);
         playgroundContainer = blobServiceClient.createBlobContainer(PLAYGROUND_CONTAINER);
+        publicAccessContainer = blobServiceClient.createBlobContainer(PUBLIC_ACCESS_CONTAINER);
+        publicAccessContainer.setAccessPolicy(PublicAccessType.CONTAINER, null);
         LOGGER.info("container " + PLAYGROUND_CONTAINER + " created successfully");
 
         LOGGER.info("Adding JSON files");
@@ -343,6 +348,7 @@
 
         // Load the data
         playgroundContainer.getBlobClient(basePath + finalFileName).uploadFromFile(filePath.toString());
+        publicAccessContainer.getBlobClient(basePath + finalFileName).uploadFromFile(filePath.toString());
         if (copyToSubLevels) {
             playgroundContainer.getBlobClient(basePath + "level1a/" + finalFileName)
                     .uploadFromFile(filePath.toString());
@@ -510,8 +516,8 @@
 
         public void executeTestFile(TestCaseContext testCaseCtx, TestFileContext ctx, Map<String, Object> variableCtx,
                 String statement, boolean isDmlRecoveryTest, ProcessBuilder pb, TestCase.CompilationUnit cUnit,
-                MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath,
-                BitSet expectedWarnings) throws Exception {
+                MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath)
+                throws Exception {
             String[] lines;
             switch (ctx.getType()) {
                 case "container":
@@ -527,7 +533,7 @@
                     break;
                 default:
                     super.executeTestFile(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit,
-                            queryCount, expectedResultFileCtxs, testFile, actualPath, expectedWarnings);
+                            queryCount, expectedResultFileCtxs, testFile, actualPath);
             }
         }
     }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
index 98f328d..2039d53 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
@@ -19,8 +19,6 @@
 package org.apache.asterix.test.sqlpp;
 
 import static org.apache.hyracks.util.file.FileUtil.canonicalize;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -69,6 +67,7 @@
 import org.apache.asterix.testframework.xml.TestGroup;
 import org.apache.hyracks.test.support.TestUtils;
 import org.junit.Assert;
+import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -151,7 +150,7 @@
             when(metadataProvider.getDefaultDataverseName()).thenReturn(dvName);
             when(metadataProvider.getConfig()).thenReturn(config);
             when(config.get(FunctionUtil.IMPORT_PRIVATE_FUNCTIONS)).thenReturn("true");
-            when(metadataProvider.findDataverse(any(DataverseName.class))).thenAnswer(new Answer<Dataverse>() {
+            when(metadataProvider.findDataverse(Mockito.<DataverseName> any())).thenAnswer(new Answer<Dataverse>() {
                 @Override
                 public Dataverse answer(InvocationOnMock invocation) {
                     Object[] args = invocation.getArguments();
@@ -160,17 +159,18 @@
                     return mockDataverse;
                 }
             });
-            when(metadataProvider.findDataset(any(DataverseName.class), anyString())).thenAnswer(new Answer<Dataset>() {
-                @Override
-                public Dataset answer(InvocationOnMock invocation) {
-                    Object[] args = invocation.getArguments();
-                    final Dataset mockDataset = mock(Dataset.class);
-                    when(mockDataset.getDataverseName()).thenReturn((DataverseName) args[0]);
-                    when(mockDataset.getDatasetName()).thenReturn((String) args[1]);
-                    return mockDataset;
-                }
-            });
-            when(metadataProvider.lookupUserDefinedFunction(any(FunctionSignature.class)))
+            when(metadataProvider.findDataset(Mockito.<DataverseName> any(), Mockito.<String> any()))
+                    .thenAnswer(new Answer<Dataset>() {
+                        @Override
+                        public Dataset answer(InvocationOnMock invocation) {
+                            Object[] args = invocation.getArguments();
+                            final Dataset mockDataset = mock(Dataset.class);
+                            when(mockDataset.getDataverseName()).thenReturn((DataverseName) args[0]);
+                            when(mockDataset.getDatasetName()).thenReturn((String) args[1]);
+                            return mockDataset;
+                        }
+                    });
+            when(metadataProvider.lookupUserDefinedFunction(Mockito.<FunctionSignature> any()))
                     .thenAnswer(new Answer<Function>() {
                         @Override
                         public Function answer(InvocationOnMock invocation) {
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query1.sqlpp
new file mode 100644
index 0000000..6953d43
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query1.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType AS { _id: bigint, c : bigint };
+CREATE DATASET    IndexDatasetA (GenericType)
+PRIMARY KEY       _id;
+CREATE DATASET    IndexDatasetB (GenericType)
+PRIMARY KEY       _id;
+CREATE DATASET    ProbeDataset (GenericType)
+PRIMARY KEY       _id;
+
+CREATE INDEX      indexA
+ON                IndexDatasetA (k : int);
+CREATE INDEX      indexB
+ON                IndexDatasetB (k : int);
+
+--                Query 1, ternary join w/ primary key on probe.
+FROM              ProbeDataset P,
+                  IndexDatasetA A,
+                  IndexDatasetB B
+WHERE             P._id /* +indexnl */ = A.k AND
+                  P._id /* +indexnl */ = B.k
+SELECT            COUNT(*);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query2.sqlpp
new file mode 100644
index 0000000..344fadd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query2.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType AS { _id: bigint, c : bigint };
+CREATE DATASET    IndexDatasetA (GenericType)
+PRIMARY KEY       _id;
+CREATE DATASET    IndexDatasetB (GenericType)
+PRIMARY KEY       _id;
+CREATE DATASET    ProbeDataset (GenericType)
+PRIMARY KEY       _id;
+
+CREATE INDEX      indexA
+ON                IndexDatasetA (k : int);
+CREATE INDEX      indexB
+ON                IndexDatasetB (k : int);
+
+--                Query 2, ternary join w/ closed field on probe.
+FROM              ProbeDataset P,
+                  IndexDatasetA A,
+                  IndexDatasetB B
+WHERE             P.c /* +indexnl */ = A.k AND
+                  P.c /* +indexnl */ = B.k
+SELECT            COUNT(*);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query3.sqlpp
new file mode 100644
index 0000000..ce0d563
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query3.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType AS { _id: bigint, c : bigint };
+CREATE DATASET    IndexDatasetA (GenericType)
+PRIMARY KEY       _id;
+CREATE DATASET    IndexDatasetB (GenericType)
+PRIMARY KEY       _id;
+CREATE DATASET    IndexDatasetC (GenericType)
+PRIMARY KEY       _id;
+CREATE DATASET    ProbeDataset (GenericType)
+PRIMARY KEY       _id;
+
+CREATE INDEX      indexA
+ON                IndexDatasetA (k : int);
+CREATE INDEX      indexB
+ON                IndexDatasetB (k : int);
+CREATE INDEX      indexC
+ON                IndexDatasetC (k : int);
+
+--                Query 3, quad-nary join w/ closed field on probe.
+FROM              ProbeDataset P,
+                  IndexDatasetA A,
+                  IndexDatasetB B,
+                  IndexDatasetC C
+WHERE             P.c /* +indexnl */ = A.k AND
+                  P.c /* +indexnl */ = B.k AND
+                  P.c /* +indexnl */ = C.k
+SELECT            COUNT(*);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query1.plan
new file mode 100644
index 0000000..671d511
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query1.plan
@@ -0,0 +1,35 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (TestDataverse.IndexDatasetB.IndexDatasetB)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$73(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (TestDataverse.IndexDatasetB.indexB)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (TestDataverse.IndexDatasetA.IndexDatasetA)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$71(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH (TestDataverse.IndexDatasetA.indexA)  |PARTITIONED|
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN (TestDataverse.ProbeDataset)  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query2.plan
new file mode 100644
index 0000000..c0f794e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query2.plan
@@ -0,0 +1,37 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (TestDataverse.IndexDatasetB.IndexDatasetB)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$73(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (TestDataverse.IndexDatasetB.indexB)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (TestDataverse.IndexDatasetA.IndexDatasetA)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$71(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH (TestDataverse.IndexDatasetA.indexA)  |PARTITIONED|
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (TestDataverse.ProbeDataset)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query3.plan
new file mode 100644
index 0000000..b94c26d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query3.plan
@@ -0,0 +1,49 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (TestDataverse.IndexDatasetC.IndexDatasetC)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$91(ASC)]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (TestDataverse.IndexDatasetC.indexC)  |PARTITIONED|
+                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (TestDataverse.IndexDatasetB.IndexDatasetB)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$89(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH (TestDataverse.IndexDatasetB.indexB)  |PARTITIONED|
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH (TestDataverse.IndexDatasetA.IndexDatasetA)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- STABLE_SORT [$$87(ASC)]  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- BTREE_SEARCH (TestDataverse.IndexDatasetA.indexA)  |PARTITIONED|
+                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- DATASOURCE_SCAN (TestDataverse.ProbeDataset)  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-no-endpoint/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-no-endpoint/test.000.ddl.sqlpp
new file mode 100644
index 0000000..d6bc862
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-no-endpoint/test.000.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+// bad case: no auth method and no endpoint provided
+drop dataset test if exists;
+CREATE EXTERNAL DATASET test(test) USING AZUREBLOB (
+("container"="playground"),
+("definition"="json-data/reviews/single-line/json"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-no-endpoint/test.099.ddl.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-no-endpoint/test.099.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/test.000.ddl.sqlpp
new file mode 100644
index 0000000..598831e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/test.000.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+CREATE EXTERNAL DATASET test(test) USING AZUREBLOB (
+("blobEndpoint"="%azureblob-endpoint%"),
+("container"="public-access"),
+("definition"="json-data/reviews/single-line/json"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/test.001.query.sqlpp
new file mode 100644
index 0000000..8ec9cc0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/test.099.ddl.sqlpp
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/test.099.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-not-allowed/test.000.ddl.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.000.ddl.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-not-allowed/test.000.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-not-allowed/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-not-allowed/test.001.query.sqlpp
new file mode 100644
index 0000000..8ec9cc0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-not-allowed/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-not-allowed/test.099.ddl.sqlpp
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-not-allowed/test.099.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-1/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-1/test.000.ddl.sqlpp
index bc6a0c4..04fb1db 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-1/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-1/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
 ("exclude"="*.json"),
 ("exclude1"="*.json")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-2/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-2/test.000.ddl.sqlpp
index a8b2c02..321d5dc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-2/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-2/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
 ("exclude"="*.json"),
 ("exclude#"="*.json")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-3/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-3/test.000.ddl.sqlpp
index 5497e13..e26f01b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-3/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-3/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
 ("exclude"="*.json"),
 ("exclude#hello"="*.json")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/both/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/both/test.000.ddl.sqlpp
index 2540c6a..428ae5e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/both/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/both/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
 ("include"="*.json"),
 ("exclude"="*.json")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-1/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-1/test.000.ddl.sqlpp
index e2883bf..e47fbd8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-1/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-1/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
 ("exclude"="*.?sv")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-2/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-2/test.000.ddl.sqlpp
index 25ae5af..85664b8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-2/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-2/test.000.ddl.sqlpp
@@ -29,9 +29,9 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
-("exclude"="data/mixed/?sv*"),
-("exclude#1"="data/mixed/json/extension*"),
-("exclude#100"="data/mixed/json/EXTENSION*")
+("exclude"="mixed-data/reviews/?sv*"),
+("exclude#1"="mixed-data/reviews/json/extension*"),
+("exclude#100"="mixed-data/reviews/json/EXTENSION*")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-3/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-3/test.000.ddl.sqlpp
index edeabbd..b127693 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-3/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-3/test.000.ddl.sqlpp
@@ -29,8 +29,8 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
-("exclude"="data/mixed/?sv*"),
-("exclude#100"="data/mixed/json*201?*")
+("exclude"="mixed-data/reviews/?sv*"),
+("exclude#100"="mixed-data/reviews/json*201?*")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-4/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-4/test.000.ddl.sqlpp
index c2ad561..b3f0bcf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-4/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-4/test.000.ddl.sqlpp
@@ -29,8 +29,8 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
-("exclude"="data/mixed/?sv*"),
-("exclude#1"="data/mixed/json*bye*")
+("exclude"="mixed-data/reviews/?sv*"),
+("exclude#1"="mixed-data/reviews/json*bye*")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-5/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-5/test.000.ddl.sqlpp
index bf4b1a0..ff12e07 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-5/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-5/test.000.ddl.sqlpp
@@ -29,8 +29,8 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
-("exclude"="data/mixed/?sv*"),
-("exclude#1"="data/mixed/json/extension/hello-world-2018.json")
+("exclude"="mixed-data/reviews/?sv*"),
+("exclude#1"="mixed-data/reviews/json/extension/hello-world-2018.json")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-6/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-6/test.000.ddl.sqlpp
index 3896068..d4804b5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-6/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-6/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
 ("exclude"="*.[abct][abcs][abcv]")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-all/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-all/test.000.ddl.sqlpp
index f0088c4..adef5d9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-all/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-all/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
 ("exclude"="*")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-1/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-1/test.000.ddl.sqlpp
index 3419631..9d31cd3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-1/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-1/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
 ("include"="*2018*.json")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-10/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-10/test.000.ddl.sqlpp
index 2aac2b2..f676230 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-10/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-10/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="csv"),
 ("header"=false),
 ("include"="*[abc][.*")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-11/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-11/test.000.ddl.sqlpp
index c54635f..acf651c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-11/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-11/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="csv"),
 ("header"=false),
 ("include"="*.[a-c][a-z][a-z**||\\\\&&--~~]")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-12/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-12/test.000.ddl.sqlpp
index 6996c56..30d3c63 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-12/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-12/test.000.ddl.sqlpp
@@ -31,7 +31,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="csv"),
 ("header"=false),
 ("include"="[][!][^]]]]*[![*a-zA--&&^$||0-9B$\\*&&]*&&[^a-b||0--9][[[")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-2/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-2/test.000.ddl.sqlpp
index d37f273..ac496ec 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-2/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-2/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
 ("include"="*201?*.json")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-3/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-3/test.000.ddl.sqlpp
index cb42507..a367c04 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-3/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-3/test.000.ddl.sqlpp
@@ -31,7 +31,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="csv"),
 ("header"=false),
 ("include"="*201?*.csv")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-4/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-4/test.000.ddl.sqlpp
index 1356218..44ce34e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-4/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-4/test.000.ddl.sqlpp
@@ -31,7 +31,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="tsv"),
 ("header"=false),
 ("include"="*201?*.tsv")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-5/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-5/test.000.ddl.sqlpp
index 52facb3..ebaa926 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-5/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-5/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
 ("include"="*bye*.json")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-6/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-6/test.000.ddl.sqlpp
index 89bbcee..baa137c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-6/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-6/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
-("include"="data/mixed/json/*EXTENSION*")
+("include"="mixed-data/reviews/json/*EXTENSION*")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-7/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-7/test.000.ddl.sqlpp
index de90340..7a569d8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-7/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-7/test.000.ddl.sqlpp
@@ -29,8 +29,8 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
-("include"="data/mixed/json/NO-EXTENSION*"),
-("include#0"="data/mixed/json/EXTENSION*")
+("include"="mixed-data/reviews/json/NO-EXTENSION*"),
+("include#0"="mixed-data/reviews/json/EXTENSION*")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-8/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-8/test.000.ddl.sqlpp
index 86a669d..75087f3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-8/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-8/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="csv"),
 ("header"=false),
 ("include"="*.[!xyt][!xyz][!xyz]")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-9/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-9/test.000.ddl.sqlpp
index 0dc8472..ec8e59a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-9/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-9/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="csv"),
 ("header"=false),
 ("include"="*.[a-c][a-z][a-z]")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-all/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-all/test.000.ddl.sqlpp
index 0255e87..6ebbe1c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-all/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-all/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
 ("include"="*")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/no-files-returned/exclude-all-files/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/no-files-returned/exclude-all-files/test.000.ddl.sqlpp
index 71130e2..0aceee0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/no-files-returned/exclude-all-files/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/no-files-returned/exclude-all-files/test.000.ddl.sqlpp
@@ -30,7 +30,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
 ("exclude"="*")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/no-files-returned/include-no-files/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/no-files-returned/include-no-files/test.000.ddl.sqlpp
index 9edc6e6..8b136cc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/no-files-returned/include-no-files/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/no-files-returned/include-no-files/test.000.ddl.sqlpp
@@ -30,7 +30,7 @@
 CREATE EXTERNAL DATASET test(test) USING %adapter% (
 %template%,
 ("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
 ("format"="json"),
 ("include"="*.notRealExtension")
 );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/select-let/select-let.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/select-let/select-let.2.query.sqlpp
new file mode 100644
index 0000000..5fbe641
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/select-let/select-let.2.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Test SELECT ... LET ... WHERE ... (no FROM clause)
+ */
+
+select value x
+let x = 2
+where x > 0
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-07-error/core-07-error.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-07-error/core-07-error.1.query.sqlpp
new file mode 100644
index 0000000..04a063c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-07-error/core-07-error.1.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Failure: GROUP BY without FROM clause
+ */
+
+SELECT x, y, COUNT(*) AS cnt
+GROUP BY x, y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.1.ddl.sqlpp
new file mode 100644
index 0000000..10f5546
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Testing group by over cross product (ASTERIXDB-2917)
+ * Expected     : SUCCESS
+ */
+
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset t1(id integer not unknown) open type primary key id;
+
+create dataset t2(id integer not unknown) open type primary key id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.2.update.sqlpp
new file mode 100644
index 0000000..7fe3777
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.2.update.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into t1 ([
+  { "id": 1, "x": "a" },
+  { "id": 2, "x": "a" },
+  { "id": 3, "x": "a" },
+  { "id": 4, "x": "b" },
+  { "id": 5, "x": "b" },
+  { "id": 6, "x": "b" }
+]);
+
+insert into t2 ([
+  { "id": 11, "y": "c" },
+  { "id": 12, "y": "c" }
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.3.query.sqlpp
new file mode 100644
index 0000000..34ec706
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select t1.x, count(*) as cnt
+from t2, t1
+group by t1.x
+order by t1.x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.1.ddl.sqlpp
new file mode 100644
index 0000000..58d2013
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.1.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType AS { _id: uuid, c: bigint };
+CREATE DATASET    IndexDatasetA (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    IndexDatasetB (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    IndexDatasetC (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    ProbeDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE INDEX      indexA
+ON                IndexDatasetA (k: int);
+CREATE INDEX      indexB
+ON                IndexDatasetB (k: int);
+CREATE INDEX      indexC
+ON                IndexDatasetC (k: int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.2.update.sqlpp
new file mode 100644
index 0000000..04b3c42
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.2.update.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE               TestDataverse;
+
+INSERT INTO       ProbeDataset [
+    { "c": 1 },
+    { "c": 2 },
+    { "c": 3 }
+];
+INSERT INTO       IndexDatasetA [
+    { "c": 1, "k": 1 },
+    { "c": 2, "k": 2 },
+    { "c": 3, "k": 4 }
+];
+INSERT INTO       IndexDatasetB [
+    { "c": 1, "k": 1 },
+    { "c": 2, "k": 2 },
+    { "c": 3, "k": 4 }
+];
+INSERT INTO       IndexDatasetC [
+    { "c": 1, "k": 1 },
+    { "c": 2, "k": 2 },
+    { "c": 3, "k": 4 }
+];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.3.query.sqlpp
new file mode 100644
index 0000000..9e65e5f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE               TestDataverse;
+
+FROM              ProbeDataset P,
+                  IndexDatasetA A,
+                  IndexDatasetB B
+WHERE             P.c /* +indexnl */ = A.k AND
+                  P.c /* +indexnl */ = B.k
+SELECT            P.c
+ORDER BY          P.c;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.4.query.sqlpp
new file mode 100644
index 0000000..f59cd69
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.4.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE               TestDataverse;
+
+FROM              ProbeDataset P,
+                  IndexDatasetA A,
+                  IndexDatasetB B,
+                  IndexDatasetC C
+WHERE             P.c /* +indexnl */ = A.k AND
+                  P.c /* +indexnl */ = B.k AND
+                  P.c /* +indexnl */ = C.k
+SELECT            P.c
+ORDER BY          P.c;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.1.ddl.sqlpp
new file mode 100644
index 0000000..0ab290f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.1.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: This test case is to verify the fix for ASTERIXDB-2947
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset t1(id uuid not unknown) open type primary key id autogenerated;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.2.update.sqlpp
new file mode 100644
index 0000000..5dcf4b9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.2.update.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into t1
+([
+  {"x":1, "y":"A" },
+  {"x":1, "y":"B" },
+  {"x":1, "y":"C" },
+
+  {"x":2, "y":"D" },
+  {"x":2, "y":"E" },
+  {"x":2, "y":"F" },
+
+  {"x":3, "y":"G" },
+  {"x":3, "y":"H" },
+  {"x":3, "y":"I" }
+]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.3.query.sqlpp
new file mode 100644
index 0000000..712b8d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: This test case is to verify the fix for ASTERIXDB-2947
+ */
+
+use test;
+
+select x, array_sort(array_agg({y})) yy
+from t1
+group by x
+having some r in array_agg({y}) satisfies r.y in ["B", "H"]
+order by x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.10.query.sqlpp
new file mode 100644
index 0000000..57545a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.10.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Test multiple window functions in the same statement
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+select c2,
+  (
+    select nth_value(c2, 3)
+      over (partition by one order by c2 range between unbounded preceding and unbounded following)
+      as nth
+    from t1 x
+  ) as q1
+from t1 y
+order by c2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/result.001.adm
new file mode 100644
index 0000000..187a8cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/result.001.adm
@@ -0,0 +1 @@
+{ "count": 100 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/select-let/select-let.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/select-let/select-let.2.adm
new file mode 100644
index 0000000..d8263ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/select-let/select-let.2.adm
@@ -0,0 +1 @@
+2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-cross-join/gby-cross-join.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-cross-join/gby-cross-join.3.adm
new file mode 100644
index 0000000..e1e3cf1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-cross-join/gby-cross-join.3.adm
@@ -0,0 +1,2 @@
+{ "cnt": 6, "x": "a" }
+{ "cnt": 6, "x": "b" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-multiple-join/btree-multiple-join.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-multiple-join/btree-multiple-join.1.adm
new file mode 100644
index 0000000..683f403
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-multiple-join/btree-multiple-join.1.adm
@@ -0,0 +1,2 @@
+{ "c": 1 }
+{ "c": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-multiple-join/btree-multiple-join.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-multiple-join/btree-multiple-join.2.adm
new file mode 100644
index 0000000..683f403
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-multiple-join/btree-multiple-join.2.adm
@@ -0,0 +1,2 @@
+{ "c": 1 }
+{ "c": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.3.adm
new file mode 100644
index 0000000..4f412c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.3.adm
@@ -0,0 +1,2 @@
+{ "yy": [ { "y": "A" }, { "y": "B" }, { "y": "C" } ], "x": 1 }
+{ "yy": [ { "y": "G" }, { "y": "H" }, { "y": "I" } ], "x": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.10.adm
new file mode 100644
index 0000000..8e98bdc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.10.adm
@@ -0,0 +1,4 @@
+{ "q1": [ { "nth": 3 }, { "nth": 3 }, { "nth": 3 }, { "nth": 3 } ], "c2": 1 }
+{ "q1": [ { "nth": 3 }, { "nth": 3 }, { "nth": 3 }, { "nth": 3 } ], "c2": 2 }
+{ "q1": [ { "nth": 3 }, { "nth": 3 }, { "nth": 3 }, { "nth": 3 } ], "c2": 3 }
+{ "q1": [ { "nth": 3 }, { "nth": 3 }, { "nth": 3 }, { "nth": 3 } ], "c2": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/window/misc_01/misc_01.10.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/window/misc_01/misc_01.10.ast
new file mode 100644
index 0000000..a6cbed7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/window/misc_01/misc_01.10.ast
@@ -0,0 +1,71 @@
+DataverseUse test
+Query:
+SELECT [
+FieldAccessor [
+  Variable [ Name=$y ]
+  Field=c2
+]
+c2
+(
+  SELECT [
+  WINDOW asterix.nth-value-impl@3[
+    (
+      SELECT ELEMENT [
+      FieldAccessor [
+        FieldAccessor [
+          Variable [ Name=#2 ]
+          Field=x
+        ]
+        Field=c2
+      ]
+      ]
+      FROM [        Variable [ Name=#1 ]
+        AS Variable [ Name=#2 ]
+      ]
+    )
+    LiteralExpr [LONG] [3]
+    FieldAccessor [
+      Variable [ Name=$x ]
+      Field=c2
+    ]
+  ]
+    AS Variable [ Name=#1 ]
+    (
+      x:=Variable [ Name=$x ]
+    )
+  OVER (
+    PARTITION BY
+      FieldAccessor [
+        Variable [ Name=$x ]
+        Field=one
+      ]
+    ORDER BY
+      FieldAccessor [
+        Variable [ Name=$x ]
+        Field=c2
+      ]
+      ASC
+    range between unbounded preceding and unbounded following exclude no others
+  )
+  nth
+  ]
+  FROM [    FunctionCall asterix.dataset@1[
+      LiteralExpr [STRING] [test.t1]
+    ]
+    AS Variable [ Name=$x ]
+  ]
+)
+q1
+]
+FROM [  FunctionCall asterix.dataset@1[
+    LiteralExpr [STRING] [test.t1]
+  ]
+  AS Variable [ Name=$y ]
+]
+Orderby
+  FieldAccessor [
+    Variable [ Name=$y ]
+    Field=c2
+  ]
+  ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_azure_blob_storage.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_azure_blob_storage.xml
index 1e302da..27ba148 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_azure_blob_storage.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_azure_blob_storage.xml
@@ -68,9 +68,20 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-dataset/azure_blob_storage/auth-methods">
-      <compilation-unit name="invalid-no-auth">
-        <output-dir compare="Text">invalid-no-auth</output-dir>
-        <expected-error>ASX1134: No authentication parameters provided</expected-error>
+      <compilation-unit name="anonymous-no-auth-public-access-allowed">
+        <output-dir compare="Text">anonymous-no-auth-public-access-allowed</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset/azure_blob_storage/auth-methods">
+      <compilation-unit name="anonymous-no-auth-public-access-not-allowed">
+        <output-dir compare="Text">anonymous-no-auth-public-access-not-allowed</output-dir>
+        <expected-error>Server failed to authenticate the request. Make sure the value of the Authorization header is formed correctly including the signature.</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset/azure_blob_storage/auth-methods">
+      <compilation-unit name="anonymous-no-auth-no-endpoint">
+        <output-dir compare="Text">anonymous-no-auth-no-endpoint</output-dir>
+        <expected-error>ASX1151: No authentication credentials provided, 'blobEndpoint' field is required for anonymous access</expected-error>
       </compilation-unit>
     </test-case>
   </test-group>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index c9e37e4..f0b20bf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -5622,6 +5622,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="group-by">
+      <compilation-unit name="gby-cross-join">
+        <output-dir compare="Text">gby-cross-join</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
       <compilation-unit name="gby-nested-01">
         <output-dir compare="Text">gby-nested-01</output-dir>
       </compilation-unit>
@@ -5674,6 +5679,12 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="group-by">
+      <compilation-unit name="core-07-error">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>GROUP BY x, y;<< Encountered "GROUP" at column 1]]></expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="group-by">
       <compilation-unit name="sugar-01">
         <output-dir compare="Text">core-01</output-dir>
       </compilation-unit>
@@ -8817,6 +8828,11 @@
         <output-dir compare="Text">query-ASTERIXDB-2696</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="quantifiers">
+      <compilation-unit name="query-ASTERIXDB-2947">
+        <output-dir compare="Text">query-ASTERIXDB-2947</output-dir>
+      </compilation-unit>
+    </test-case>
     <!--
         <test-case FilePath="quantifiers">
           <compilation-unit name="everysat_02">
diff --git a/asterixdb/asterix-common/pom.xml b/asterixdb/asterix-common/pom.xml
index 2d8bec2..4baf910 100644
--- a/asterixdb/asterix-common/pom.xml
+++ b/asterixdb/asterix-common/pom.xml
@@ -144,7 +144,7 @@
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
+      <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 4078ea2..e80dbfb 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -235,6 +235,7 @@
     CANNOT_DROP_OBJECT_DEPENDENT_EXISTS(1148),
     ILLEGAL_FUNCTION_RECURSION(1149),
     ILLEGAL_FUNCTION_USE(1150),
+    NO_AUTH_PROVIDED_ENDPOINT_REQUIRED_FOR_ANONYMOUS_ACCESS(1151),
 
     // Feed errors
     DATAFLOW_ILLEGAL_STATE(3001),
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 18898bb..cd4a289 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -237,6 +237,7 @@
 1148 = Cannot drop %1$s %2$s being used by %3$s %4$s
 1149 = Illegal function recursion
 1150 = Illegal use of function %1$s
+1151 = No authentication credentials provided, '%1$s' field is required for anonymous access
 
 # Feed Errors
 3001 = Illegal state.
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
index 843cb21..779bfd8 100644
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
@@ -219,7 +219,7 @@
 
             @Override
             public Void answer(InvocationOnMock invocation) throws Throwable {
-                List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(0, List.class);
+                List<ILSMDiskComponent> mergedComponents = invocation.getArgument(0);
                 mergedComponents.forEach(component -> {
                     try {
                         resultComponentIDs.add(component.getId());
diff --git a/asterixdb/asterix-dashboard/pom.xml b/asterixdb/asterix-dashboard/pom.xml
index b8ac630..2b10a4e 100644
--- a/asterixdb/asterix-dashboard/pom.xml
+++ b/asterixdb/asterix-dashboard/pom.xml
@@ -66,7 +66,7 @@
           <plugin>
             <groupId>com.github.eirslett</groupId>
             <artifactId>frontend-maven-plugin</artifactId>
-            <version>1.6</version>
+            <version>1.11.0</version>
             <configuration>
               <nodeVersion>v10.3.0</nodeVersion>
               <npmVersion>6.1.0</npmVersion>
@@ -182,4 +182,26 @@
       <id>skip-dashboard</id>
     </profile>
   </profiles>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>touch-3rdparty</id>
+            <phase>${skip-npm-touch.stage}</phase>
+            <configuration>
+              <target>
+                <touch file="${basedir}/target/dashboard/static/3rdpartylicenses.txt" mkdirs="true"/>
+              </target>
+            </configuration>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
 </project>
diff --git a/asterixdb/asterix-doc/pom.xml b/asterixdb/asterix-doc/pom.xml
index c58d983..925beba 100644
--- a/asterixdb/asterix-doc/pom.xml
+++ b/asterixdb/asterix-doc/pom.xml
@@ -191,10 +191,10 @@
                 <id>extract-rr</id>
                 <phase>generate-sources</phase>
                 <configuration>
-                  <tasks>
+                  <target>
                     <echo message="Extracting rr" />
                     <unzip src="${project.build.directory}/rr-1.62-java8.zip" dest="${project.build.directory}" />
-                  </tasks>
+                  </target>
                 </configuration>
                 <goals>
                   <goal>run</goal>
@@ -204,13 +204,13 @@
                 <id>extract-diagrams</id>
                 <phase>process-resources</phase>
                 <configuration>
-                  <tasks>
+                  <target>
                     <echo message="Extracting diagrams" />
                     <unzip src="${project.build.directory}/railroads.zip" dest="${project.build.directory}" />
                       <copy todir="${project.build.directory}/site/images/diagrams/">
                       <fileset dir="${project.build.directory}/diagram/"/>
                     </copy>
-                  </tasks>
+                  </target>
                 </configuration>
                 <goals>
                   <goal>run</goal>
diff --git a/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf b/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
index cf7d4a5..fd14252 100644
--- a/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
+++ b/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
@@ -52,9 +52,9 @@
 
 Query ::= (Expr | Selection)
 
-Selection ::= WithClause? QueryBlock UnionOption* OrderByClause? ( LimitClause | OffsetClause )?
+Selection ::= (WithClause | LetClause)? QueryBlock UnionOption* OrderByClause? ( LimitClause | OffsetClause )?
 
-QueryBlock ::= SelectClause StreamGenerator?
+QueryBlock ::= SelectClause ( ( LetClause WhereClause? ) | StreamGenerator )?
              | StreamGenerator SelectClause
 
 StreamGenerator::= FromClause LetClause? WhereClause? (GroupByClause LetClause? HavingClause?)?
diff --git a/asterixdb/asterix-external-data/pom.xml b/asterixdb/asterix-external-data/pom.xml
index 9676da6..7b8269c 100644
--- a/asterixdb/asterix-external-data/pom.xml
+++ b/asterixdb/asterix-external-data/pom.xml
@@ -287,7 +287,7 @@
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
+      <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
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 8e94263..7872cef 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
@@ -899,7 +899,7 @@
             String blobEndpoint = configuration.get(BLOB_ENDPOINT_FIELD_NAME);
             String endpointSuffix = configuration.get(ENDPOINT_SUFFIX_FIELD_NAME);
 
-            // Constructor the connection string
+            // Construct the connection string
             // Connection string format: name1=value1;name2=value2;....
             StringBuilder connectionStringBuilder = new StringBuilder();
             BlobServiceClientBuilder builder = new BlobServiceClientBuilder();
@@ -926,17 +926,12 @@
                 if (authMethodFound) {
                     throw new CompilationException(ErrorCode.ONLY_SINGLE_AUTHENTICATION_IS_ALLOWED);
                 }
-                authMethodFound = true;
                 // account name + shared access token
                 connectionStringBuilder.append(CONNECTION_STRING_ACCOUNT_NAME).append("=").append(accountName)
                         .append(";").append(CONNECTION_STRING_SHARED_ACCESS_SIGNATURE).append("=")
                         .append(sharedAccessSignature).append(";");
             }
 
-            if (!authMethodFound) {
-                throw new CompilationException(ErrorCode.NO_AUTH_METHOD_PROVIDED);
-            }
-
             // Add blobEndpoint and endpointSuffix if present, adjust any '/' as needed
             if (blobEndpoint != null) {
                 connectionStringBuilder.append(CONNECTION_STRING_BLOB_ENDPOINT).append("=").append(blobEndpoint)
@@ -954,6 +949,12 @@
                 }
             }
 
+            // No credentials or endpoint provided
+            if (connectionStringBuilder.length() == 0) {
+                throw new CompilationException(ErrorCode.NO_AUTH_PROVIDED_ENDPOINT_REQUIRED_FOR_ANONYMOUS_ACCESS,
+                        BLOB_ENDPOINT_FIELD_NAME);
+            }
+
             try {
                 return builder.connectionString(connectionStringBuilder.toString()).buildClient();
             } catch (Exception ex) {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
index 6d8e52e..bf3e227 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
@@ -117,8 +117,11 @@
         Map<VariableExpr, Set<? extends Scope.SymbolAnnotation>> liveAnnotatedVars =
                 scopeChecker.getCurrentScope().getLiveVariables();
         Set<VariableExpr> liveVars = liveAnnotatedVars.keySet();
-        Set<VariableExpr> liveContextVars = Scope.findVariablesAnnotatedBy(liveVars,
-                SqlppVariableAnnotation.CONTEXT_VARIABLE, liveAnnotatedVars, winExpr.getSourceLocation());
+
+        Map<VariableExpr, Set<? extends Scope.SymbolAnnotation>> localAnnotatedVars =
+                scopeChecker.getCurrentScope().getLiveVariables(scopeChecker.getPrecedingScope());
+        Set<VariableExpr> liveContextVars = Scope.findVariablesAnnotatedBy(localAnnotatedVars.keySet(),
+                SqlppVariableAnnotation.CONTEXT_VARIABLE, localAnnotatedVars, winExpr.getSourceLocation());
 
         List<Pair<Expression, Identifier>> winFieldList = winExpr.getWindowFieldList();
         Map<VariableExpr, Identifier> winVarFieldMap =
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index 0aa9b58..2d3fcc5 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -3960,6 +3960,7 @@
 }
 {
   (
+    (
      selectClause = SelectClause() { startSrcLoc = selectClause.getSourceLocation(); }
      (
         (
@@ -3967,6 +3968,14 @@
           (
               fromLetClauses = LetClause()
           )?
+          (whereClause = WhereClause())?
+          (
+            groupbyClause = GroupbyClause()
+            (
+                gbyLetClauses = LetClause()
+            )?
+            (havingClause = HavingClause())?
+          )?
         )
         |
         (
@@ -3986,17 +3995,12 @@
             fromTerms.add(new FromTerm(listExpr, fromVar, null, new ArrayList<AbstractBinaryCorrelateClause>()));
             fromClause = new FromClause(fromTerms);
           }
+          (whereClause = WhereClause())?
         )
      )?
-     (whereClause = WhereClause())?
-     (
-        groupbyClause = GroupbyClause()
-        (
-            gbyLetClauses = LetClause()
-        )?
-        (havingClause = HavingClause())?
-     )?
+    )
     |
+    (
      fromClause = FromClause() { startSrcLoc = fromClause.getSourceLocation(); }
      (
         fromLetClauses = LetClause()
@@ -4010,6 +4014,7 @@
         (havingClause = HavingClause())?
      )?
      selectClause = SelectClause()
+    )
   )
   {
     if (fromLetClauses != null) {
diff --git a/asterixdb/asterix-metadata/pom.xml b/asterixdb/asterix-metadata/pom.xml
index 1e2bf53..248d2e0 100644
--- a/asterixdb/asterix-metadata/pom.xml
+++ b/asterixdb/asterix-metadata/pom.xml
@@ -155,7 +155,7 @@
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
+      <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/asterixdb/asterix-om/pom.xml b/asterixdb/asterix-om/pom.xml
index 4ee8e83..f3a1f08 100644
--- a/asterixdb/asterix-om/pom.xml
+++ b/asterixdb/asterix-om/pom.xml
@@ -38,6 +38,28 @@
     <root.dir>${basedir}/..</root.dir>
   </properties>
 
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+          <configuration>
+            <parallel>all</parallel>
+            <forkCount>4</forkCount>
+            <reuseForks>false</reuseForks>
+            <threadCount>1</threadCount>
+            <argLine>
+              -enableassertions -Xmx${test.heap.size}m
+              -Dfile.encoding=UTF-8
+              -DrunSlowAQLTests=${runSlowAQLTests}
+              -Xdebug
+              ${coverageArgLine}
+            </argLine>
+          </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
   <dependencies>
     <dependency>
       <groupId>org.apache.asterix</groupId>
@@ -61,7 +83,7 @@
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
+      <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/asterixdb/asterix-runtime/pom.xml b/asterixdb/asterix-runtime/pom.xml
index 6b8bf71..61e3086 100644
--- a/asterixdb/asterix-runtime/pom.xml
+++ b/asterixdb/asterix-runtime/pom.xml
@@ -139,7 +139,7 @@
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
+      <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/asterixdb/asterix-server/pom.xml b/asterixdb/asterix-server/pom.xml
index ce943ad..0176ef8 100644
--- a/asterixdb/asterix-server/pom.xml
+++ b/asterixdb/asterix-server/pom.xml
@@ -254,6 +254,7 @@
                 <aliasUrl>http://www.apache.org/licenses/LICENSE-2.0</aliasUrl>
                 <aliasUrl>https://www.apache.org/licenses/LICENSE-2.0.txt</aliasUrl>
                 <aliasUrl>http://www.apache.org/licenses/LICENSE-2.0.html</aliasUrl>
+                <aliasUrl>https://www.apache.org/licenses/LICENSE-2.0.html</aliasUrl>
                 <aliasUrl>https://aws.amazon.com/apache2.0</aliasUrl>
               </aliasUrls>
               <metric>1</metric>
diff --git a/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestCaseContext.java b/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestCaseContext.java
index 859e0bf..c73434d 100644
--- a/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestCaseContext.java
+++ b/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestCaseContext.java
@@ -21,6 +21,7 @@
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.BitSet;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -37,77 +38,18 @@
 
 public class TestCaseContext {
 
-    /**
-     * For specifying the desired output formatting of results.
-     */
-    public enum OutputFormat {
-        NONE("", ""),
-        ADM("adm", "application/x-adm"),
-        LOSSLESS_JSON("json", "application/json; lossless=true"),
-        CLEAN_JSON("json", "application/json"),
-        CSV("csv", "text/csv"),
-        CSV_HEADER("csv-header", "text/csv; header=present"),
-        AST("ast", "application/x-ast"),
-        PLAN("plan", "application/x-plan"),
-        BINARY("", "application/octet-stream");
-
-        private final String extension;
-        private final String mimetype;
-
-        OutputFormat(String ext, String mime) {
-            this.extension = ext;
-            this.mimetype = mime;
-        }
-
-        public String extension() {
-            return extension;
-        }
-
-        public String mimeType() {
-            return mimetype;
-        }
-
-        //
-        public static OutputFormat forCompilationUnit(CompilationUnit cUnit) {
-            switch (cUnit.getOutputDir().getCompare()) {
-                case TEXT:
-                    return OutputFormat.ADM;
-                case LOSSLESS_JSON:
-                    return OutputFormat.LOSSLESS_JSON;
-                case CLEAN_JSON:
-                    return OutputFormat.CLEAN_JSON;
-                case CSV:
-                    return OutputFormat.CSV;
-                case CSV_HEADER:
-                    return OutputFormat.CSV_HEADER;
-                case BINARY:
-                    return OutputFormat.BINARY;
-                case INSPECT:
-                case IGNORE:
-                    return OutputFormat.NONE;
-                case AST:
-                    return OutputFormat.AST;
-                case PLAN:
-                    return OutputFormat.PLAN;
-                default:
-                    assert false : "Unknown ComparisonEnum!";
-                    return OutputFormat.NONE;
-            }
-        }
-    }
-
     public static final String DEFAULT_TESTSUITE_XML_NAME = "testsuite.xml";
     public static final String ONLY_TESTSUITE_XML_NAME = "only.xml";
     public static final String DEFAULT_REPEATED_TESTSUITE_XML_NAME = "repeatedtestsuite.xml";
 
     private File tsRoot;
-
     private TestSuite testSuite;
-
     private TestGroup[] testGroups;
-
     private TestCase testCase;
     private Map<String, Object> kv;
+    public int numOfErrors;
+    public List<String> expectedErrors;
+    public BitSet expectedWarnings;
 
     public TestCaseContext(File tsRoot, TestSuite testSuite, TestGroup[] testGroups, TestCase testCase) {
         this.tsRoot = tsRoot;
@@ -234,16 +176,16 @@
     }
 
     public static class Builder {
-        private final boolean m_doSlow;
-        private final Pattern m_re;
+        private final boolean doSlow;
+        private final Pattern re;
 
         public Builder() {
-            m_doSlow = System.getProperty("runSlowAQLTests", "false").equals("true");
-            String re = System.getProperty("testre");
-            if (re == null) {
-                m_re = null;
+            doSlow = System.getProperty("runSlowAQLTests", "false").equals("true");
+            String testre = System.getProperty("testre");
+            if (testre == null) {
+                this.re = null;
             } else {
-                m_re = Pattern.compile(re);
+                this.re = Pattern.compile(testre);
             }
         }
 
@@ -280,13 +222,13 @@
         private void addContexts(File tsRoot, TestSuite ts, List<TestGroup> tgPath, List<TestCaseContext> tccs) {
             TestGroup tg = tgPath.get(tgPath.size() - 1);
             for (TestCase tc : tg.getTestCase()) {
-                if (m_doSlow || tc.getCategory() != CategoryEnum.SLOW) {
+                if (doSlow || tc.getCategory() != CategoryEnum.SLOW) {
                     boolean matches = false;
-                    if (m_re != null) {
+                    if (re != null) {
                         // Check all compilation units for matching
                         // name. If ANY match, add the test.
                         for (TestCase.CompilationUnit cu : tc.getCompilationUnit()) {
-                            if (m_re.matcher(cu.getName()).find()) {
+                            if (re.matcher(cu.getName()).find()) {
                                 matches = true;
                                 break;
                             }
@@ -304,4 +246,63 @@
             addContexts(tsRoot, ts, tgPath, tg.getTestGroup(), tccs);
         }
     }
+
+    /**
+     * For specifying the desired output formatting of results.
+     */
+    public enum OutputFormat {
+        NONE("", ""),
+        ADM("adm", "application/x-adm"),
+        LOSSLESS_JSON("json", "application/json; lossless=true"),
+        CLEAN_JSON("json", "application/json"),
+        CSV("csv", "text/csv"),
+        CSV_HEADER("csv-header", "text/csv; header=present"),
+        AST("ast", "application/x-ast"),
+        PLAN("plan", "application/x-plan"),
+        BINARY("", "application/octet-stream");
+
+        private final String extension;
+        private final String mimetype;
+
+        OutputFormat(String ext, String mime) {
+            this.extension = ext;
+            this.mimetype = mime;
+        }
+
+        public String extension() {
+            return extension;
+        }
+
+        public String mimeType() {
+            return mimetype;
+        }
+
+        //
+        public static OutputFormat forCompilationUnit(CompilationUnit cUnit) {
+            switch (cUnit.getOutputDir().getCompare()) {
+                case TEXT:
+                    return OutputFormat.ADM;
+                case LOSSLESS_JSON:
+                    return OutputFormat.LOSSLESS_JSON;
+                case CLEAN_JSON:
+                    return OutputFormat.CLEAN_JSON;
+                case CSV:
+                    return OutputFormat.CSV;
+                case CSV_HEADER:
+                    return OutputFormat.CSV_HEADER;
+                case BINARY:
+                    return OutputFormat.BINARY;
+                case INSPECT:
+                case IGNORE:
+                    return OutputFormat.NONE;
+                case AST:
+                    return OutputFormat.AST;
+                case PLAN:
+                    return OutputFormat.PLAN;
+                default:
+                    assert false : "Unknown ComparisonEnum!";
+                    return OutputFormat.NONE;
+            }
+        }
+    }
 }
diff --git a/asterixdb/asterix-transactions/pom.xml b/asterixdb/asterix-transactions/pom.xml
index 511b357..1d7293c 100644
--- a/asterixdb/asterix-transactions/pom.xml
+++ b/asterixdb/asterix-transactions/pom.xml
@@ -110,7 +110,7 @@
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
+      <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 508eb76..29dedf7 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -230,20 +230,21 @@
     @Override
     public synchronized void delete(String relativePath) throws HyracksDataException {
         FileReference resourceFile = getLocalResourceFileByName(ioManager, relativePath);
-        if (resourceFile.getFile().exists()) {
-            if (isReplicationEnabled) {
-                createReplicationJob(ReplicationOperation.DELETE, resourceFile);
+        try {
+            if (resourceFile.getFile().exists()) {
+                if (isReplicationEnabled) {
+                    createReplicationJob(ReplicationOperation.DELETE, resourceFile);
+                }
+                final LocalResource localResource = readLocalResource(resourceFile.getFile());
+                IoUtil.delete(resourceFile);
+                // delete all checkpoints
+                indexCheckpointManagerProvider.get(DatasetResourceReference.of(localResource)).delete();
+            } else {
+                throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.RESOURCE_DOES_NOT_EXIST,
+                        relativePath);
             }
-            final LocalResource localResource = readLocalResource(resourceFile.getFile());
-            // Invalidate before deleting the file just in case file deletion throws some exception.
-            // Since it's just a cache invalidation, it should not affect correctness.
+        } finally {
             resourceCache.invalidate(relativePath);
-            IoUtil.delete(resourceFile);
-            // delete all checkpoints
-            indexCheckpointManagerProvider.get(DatasetResourceReference.of(localResource)).delete();
-        } else {
-            throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.RESOURCE_DOES_NOT_EXIST,
-                    relativePath);
         }
     }
 
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index a930643..8b769f4 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -46,7 +46,8 @@
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <file.encoding>UTF-8</file.encoding>
     <jvm.extraargs />
-    <source.jdk.version>1.8</source.jdk.version>
+    <source.jdk.version>11</source.jdk.version>
+    <target.jdk.version>11</target.jdk.version>
     <javac.xlint.value>all</javac.xlint.value>
     <source-format.goal>format</source-format.goal>
     <source-format.skip>false</source-format.skip>
@@ -78,6 +79,7 @@
     <prepare-env.stage>none</prepare-env.stage>
     <shim.stage>none</shim.stage>
     <pytestlib.stage>none</pytestlib.stage>
+    <skip-npm-touch.stage>none</skip-npm-touch.stage>
 
     <!-- Versions under dependencymanagement or used in many projects via properties -->
     <algebricks.version>0.3.6-SNAPSHOT</algebricks.version>
@@ -284,7 +286,7 @@
                 <condition property="skipPomEnforcement">
                   <not><equals arg1="${project.packaging}" arg2="pom"/></not>
                 </condition>
-                <echo message="will enforce non-existence of test source dir due to ${project.packaging} packaging" unless:set="skipPomEnforcement"/>
+                <echo level="info" message="will enforce non-existence of test source dir due to ${project.packaging} packaging" unless:set="skipPomEnforcement"/>
               </target>
             </configuration>
           </execution>
@@ -322,6 +324,20 @@
                 <requireMavenVersion>
                   <version>[3.3.9,)</version>
                 </requireMavenVersion>
+                <requireJavaVersion>
+                  <version>[11,)</version>
+                </requireJavaVersion>
+              </rules>
+            </configuration>
+          </execution>
+          <execution>
+            <id>ensure-no-duplicate-deps</id>
+            <goals>
+              <goal>enforce</goal>
+            </goals>
+            <configuration>
+              <rules>
+                <banDuplicatePomDependencyVersions/>
               </rules>
             </configuration>
           </execution>
@@ -528,7 +544,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-surefire-plugin</artifactId>
-          <version>2.21.0</version>
+          <version>3.0.0-M5</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
@@ -551,7 +567,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-antrun-plugin</artifactId>
-          <version>1.8</version>
+          <version>3.0.0</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
@@ -576,7 +592,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-dependency-plugin</artifactId>
-          <version>3.1.1</version>
+          <version>3.1.2</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
@@ -630,6 +646,11 @@
           <artifactId>exec-maven-plugin</artifactId>
           <version>1.6.0</version>
         </plugin>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-enforcer-plugin</artifactId>
+          <version>3.0.0-M3</version>
+        </plugin>
       </plugins>
     </pluginManagement>
   </build>
@@ -691,6 +712,18 @@
       </properties>
     </profile>
     <profile>
+      <id>skip-npm</id>
+      <activation>
+        <property>
+          <name>skip.npm</name>
+          <value>true</value>
+        </property>
+      </activation>
+      <properties>
+          <skip-npm-touch.stage>process-sources</skip-npm-touch.stage>
+      </properties>
+    </profile>
+    <profile>
       <id>invalid-tests</id>
       <properties>
         <invalid.tests />
@@ -705,7 +738,7 @@
         </property>
       </activation>
       <properties>
-        <test.heap.size>2048</test.heap.size>
+        <test.heap.size>4096</test.heap.size>
       </properties>
     </profile>
 
@@ -718,7 +751,7 @@
         </property>
       </activation>
       <properties>
-        <test.heap.size>3072</test.heap.size>
+        <test.heap.size>4096</test.heap.size>
       </properties>
     </profile>
     <profile>
@@ -867,40 +900,6 @@
         <failIfNoTests>false</failIfNoTests>
       </properties>
     </profile>
-    <profile>
-      <id>java8</id>
-      <activation>
-        <jdk>[1.8,)</jdk>
-      </activation>
-      <properties>
-        <target.jdk.version>8</target.jdk.version>
-      </properties>
-    </profile>
-    <profile>
-      <id>java8-bootclasspath</id>
-      <activation>
-        <property>
-          <name>java8-bootclasspath</name>
-        </property>
-      </activation>
-      <properties>
-        <target.jdk.version>8</target.jdk.version>
-      </properties>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-compiler-plugin</artifactId>
-            <configuration>
-              <compilerArgs combine.children="append">
-                <arg>-bootclasspath</arg>
-                <arg>${java8-bootclasspath}</arg>
-              </compilerArgs>
-            </configuration>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
   </profiles>
 
   <modules>
diff --git a/asterixdb/src/main/appended-resources/supplemental-models.xml b/asterixdb/src/main/appended-resources/supplemental-models.xml
index c352fe5..4849544 100644
--- a/asterixdb/src/main/appended-resources/supplemental-models.xml
+++ b/asterixdb/src/main/appended-resources/supplemental-models.xml
@@ -251,8 +251,8 @@
       <properties>
         <!-- fastutil is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
         <!-- see https://github.com/vigna/fastutil/blob/8.5.4/LICENSE-2.0 -->
-        <license.ignoreMissingEmbeddedLicense>8.5.4</license.ignoreMissingEmbeddedLicense>
-        <license.ignoreMissingEmbeddedNotice>8.5.4</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreMissingEmbeddedLicense>8.3.0,8.5.4</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>8.3.0,8.5.4</license.ignoreMissingEmbeddedNotice>
       </properties>
     </project>
   </supplement>
@@ -287,9 +287,10 @@
       <properties>
         <!-- snappy-java is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
         <!-- license override not needed, ALv2 is specified in its pom.xml -->
-        <!-- see https://github.com/xerial/snappy-java/blob/1.1.7.1/LICENSE -->
-        <license.ignoreMissingEmbeddedLicense>1.1.7.1</license.ignoreMissingEmbeddedLicense>
-        <license.ignoreMissingEmbeddedNotice>1.1.7.1</license.ignoreMissingEmbeddedNotice>
+        <!-- see https://github.com/xerial/snappy-java/blob/1.1.8.4/LICENSE -->
+        <license.ignoreMissingEmbeddedLicense>1.1.7.1,1.1.8.4</license.ignoreMissingEmbeddedLicense>
+        <license.ignoreMissingEmbeddedNotice>1.1.7.1,1.1.8.4</license.ignoreMissingEmbeddedNotice>
+        <license.ignoreLicenseOverride>1.1.7.1</license.ignoreLicenseOverride>
       </properties>
     </project>
   </supplement>
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml b/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
index dbb9363..0574afc 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
@@ -68,26 +68,22 @@
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <version>1.10.19</version>
+      <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
    <dependency>
       <groupId>org.powermock</groupId>
-      <artifactId>powermock-api-mockito</artifactId>
-      <version>1.6.6</version>
+      <artifactId>powermock-api-mockito2</artifactId>
       <scope>test</scope>
    </dependency>
    <dependency>
       <groupId>org.powermock</groupId>
       <artifactId>powermock-core</artifactId>
-      <version>1.6.6</version>
       <scope>test</scope>
    </dependency>
    <dependency>
       <groupId>org.powermock</groupId>
       <artifactId>powermock-module-junit4</artifactId>
-      <version>1.6.6</version>
       <scope>test</scope>
    </dependency>
    <dependency>
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
index 18aa104..b568417 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
@@ -41,7 +41,7 @@
 import org.powermock.modules.junit4.PowerMockRunner;
 
 @RunWith(PowerMockRunner.class)
-@PowerMockIgnore("javax.management.*")
+@PowerMockIgnore({ "javax.management.*", "javax.xml.*", "com.sun.org.apache.xerces.*", "org.w3c.*", "org.xml.*" })
 @PrepareForTest({ SequentialFirstRuleCheckFixpointRuleController.class, AbstractLogicalOperator.class })
 public class SequentialFirstRuleCheckFixpointRuleControllerTest {
 
diff --git a/hyracks-fullstack/hyracks/hyracks-api/pom.xml b/hyracks-fullstack/hyracks/hyracks-api/pom.xml
index 2fab94a..6954556 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-api/pom.xml
@@ -66,7 +66,7 @@
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
+      <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksThrowingConsumer.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksThrowingConsumer.java
index b677132..765f9d7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksThrowingConsumer.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksThrowingConsumer.java
@@ -18,9 +18,25 @@
  */
 package org.apache.hyracks.api.util;
 
+import java.util.function.Consumer;
+
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
+import com.google.common.util.concurrent.UncheckedExecutionException;
+
 @FunctionalInterface
 public interface HyracksThrowingConsumer<V> {
-    void process(V value) throws HyracksDataException;
+    void accept(V value) throws HyracksDataException;
+
+    @SuppressWarnings("Duplicates")
+    static <T> Consumer<T> asUnchecked(HyracksThrowingConsumer<T> consumer) {
+        return input -> {
+            try {
+                consumer.accept(input);
+            } catch (HyracksDataException e) {
+                throw new UncheckedExecutionException(e);
+            }
+        };
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
index 0f1be96..26f411a 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
@@ -88,7 +88,7 @@
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
+      <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
index 8c543f3..e4dc2ef 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
@@ -116,7 +116,7 @@
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
+      <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-http/pom.xml b/hyracks-fullstack/hyracks/hyracks-http/pom.xml
index 0c94743..9431d95 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-http/pom.xml
@@ -89,7 +89,7 @@
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
+      <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
index c61e0d2..835cd54 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
@@ -45,6 +45,7 @@
 import org.apache.hyracks.http.server.FormUrlEncodedRequest;
 import org.apache.hyracks.http.server.HttpServer;
 import org.apache.hyracks.util.ThrowingConsumer;
+import org.apache.hyracks.util.ThrowingFunction;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -191,16 +192,23 @@
         return i < 0 ? uri : uri.substring(0, i);
     }
 
-    public static void handleStreamInterruptibly(CloseableHttpResponse response,
+    public static void consumeStreamInterruptibly(CloseableHttpResponse response,
             ThrowingConsumer<Reader> streamProcessor, ExecutorService executor, Supplier<String> descriptionSupplier)
+            throws InterruptedException, ExecutionException, IOException {
+        processStreamInterruptibly(response, ThrowingConsumer.asFunction(streamProcessor), executor,
+                descriptionSupplier);
+    }
+
+    public static <T> T processStreamInterruptibly(CloseableHttpResponse response,
+            ThrowingFunction<Reader, T> streamProcessor, ExecutorService executor, Supplier<String> descriptionSupplier)
             throws IOException, InterruptedException, ExecutionException {
         // we have to consume the stream in a separate thread, as it not stop on interrupt; we need to
         // instead close the connection to achieve the interrupt
         String description = descriptionSupplier.get();
-        Future<Void> readFuture = executor.submit(() -> {
+        Future<T> readFuture = executor.submit(() -> {
             Thread.currentThread().setName(description);
             InputStreamReader reader = new InputStreamReader(response.getEntity().getContent(), StandardCharsets.UTF_8);
-            streamProcessor.process(new Reader() {
+            return streamProcessor.process(new Reader() {
                 @Override
                 public int read(char[] cbuf, int off, int len) throws IOException {
                     return reader.read(cbuf, off, len);
@@ -213,10 +221,9 @@
                     LOGGER.debug("ignoring close on {}", reader);
                 }
             });
-            return null;
         });
         try {
-            readFuture.get();
+            return readFuture.get();
         } catch (InterruptedException ex) { // NOSONAR -- interrupt or rethrow
             response.close();
             try {
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/test/http/HttpTestUtil.java b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/test/http/HttpTestUtil.java
index af5e3e3..4debe7f 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/test/http/HttpTestUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/test/http/HttpTestUtil.java
@@ -21,16 +21,16 @@
 import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryPoolMXBean;
 import java.lang.management.MemoryType;
-import java.lang.reflect.Field;
 import java.util.List;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.ByteBufAllocatorMetric;
+import io.netty.buffer.ByteBufAllocatorMetricProvider;
 import io.netty.buffer.PooledByteBufAllocator;
-import io.netty.util.internal.PlatformDependent;
 
 public class HttpTestUtil {
     private static final Logger LOGGER = LogManager.getLogger();
@@ -66,14 +66,12 @@
         report.append("---------------------------- Netty ----------------------------");
         report.append('\n');
         try {
-            Field field = PlatformDependent.class.getDeclaredField("DIRECT_MEMORY_COUNTER");
-            field.setAccessible(true);
-            AtomicLong usedDirectMemory = (AtomicLong) field.get(null);
-            long used = usedDirectMemory.get();
-            report.append("Current PlatformDependent.DIRECT_MEMORY_COUNTER: ");
+            ByteBufAllocatorMetric memUsage = ((ByteBufAllocatorMetricProvider) ByteBufAllocator.DEFAULT).metric();
+            long used = memUsage.usedDirectMemory();
+            report.append("Current default allocator direct buffer usage: ");
             report.append(used);
             report.append('\n');
-            report.append("Maximum PlatformDependent.DIRECT_MEMORY_COUNTER: ");
+            report.append("Maximum default allocator direct buffer usage: ");
             maxMemUsage = Math.max(maxMemUsage, used);
             report.append(maxMemUsage);
             report.append('\n');
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java
index 89993fd..2206621 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java
@@ -41,13 +41,19 @@
 import java.util.stream.Collectors;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.mutable.MutableBoolean;
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hyracks.maven.license.project.LicensedProjects;
 import org.apache.hyracks.maven.license.project.Project;
 import org.apache.maven.artifact.Artifact;
+import org.apache.maven.artifact.DefaultArtifact;
+import org.apache.maven.artifact.handler.ArtifactHandler;
+import org.apache.maven.artifact.handler.DefaultArtifactHandler;
 import org.apache.maven.artifact.repository.ArtifactRepository;
+import org.apache.maven.artifact.resolver.ArtifactResolutionRequest;
+import org.apache.maven.artifact.resolver.ArtifactResolutionResult;
 import org.apache.maven.artifact.resolver.ArtifactResolver;
 import org.apache.maven.execution.MavenSession;
 import org.apache.maven.model.License;
@@ -121,6 +127,9 @@
     @Parameter
     protected boolean failOnWarning;
 
+    @Parameter
+    protected List<String> extraDependencies = new ArrayList<>();
+
     private Map<String, MavenProject> projectCache = new HashMap<>();
 
     private Map<String, Model> supplementModels;
@@ -393,24 +402,47 @@
     private void gatherProjectDependencies(MavenProject project,
             Map<MavenProject, List<Pair<String, String>>> dependencyLicenseMap,
             Map<String, MavenProject> dependencyGavMap) throws ProjectBuildingException {
-        final Set dependencyArtifacts = project.getArtifacts();
+        final Set<Artifact> dependencyArtifacts = project.getArtifacts();
         if (dependencyArtifacts != null) {
-            for (Object depArtifactObj : dependencyArtifacts) {
-                final Artifact depArtifact = (Artifact) depArtifactObj;
-                if (!excludedScopes.contains(depArtifact.getScope())) {
-                    MavenProject dep = resolveDependency(depArtifact);
-                    dep.setArtifact(depArtifact);
-                    dependencyGavMap.put(toGav(dep), dep);
-                    List<Pair<String, String>> licenseUrls = new ArrayList<>();
-                    for (Object license : dep.getLicenses()) {
-                        final License license1 = (License) license;
-                        String url = license1.getUrl() != null ? license1.getUrl()
-                                : (license1.getName() != null ? license1.getName() : "LICENSE_EMPTY_NAME_URL");
-                        licenseUrls.add(new ImmutablePair<>(url, license1.getName()));
-                    }
-                    dependencyLicenseMap.put(dep, licenseUrls);
+            for (Artifact depArtifact : dependencyArtifacts) {
+                processArtifact(depArtifact, dependencyLicenseMap, dependencyGavMap);
+            }
+        }
+        for (String gav : extraDependencies) {
+            ArtifactHandler handler = new DefaultArtifactHandler("jar");
+            String[] gavParts = StringUtils.split(gav, ':');
+            Artifact manualDep = new DefaultArtifact(gavParts[0], gavParts[1], gavParts[2], Artifact.SCOPE_COMPILE,
+                    "jar", null, handler);
+            processArtifact(manualDep, dependencyLicenseMap, dependencyGavMap);
+        }
+    }
+
+    private void processArtifact(Artifact depArtifact,
+            Map<MavenProject, List<Pair<String, String>>> dependencyLicenseMap,
+            Map<String, MavenProject> dependencyGavMap) throws ProjectBuildingException {
+        if (!excludedScopes.contains(depArtifact.getScope())) {
+            MavenProject dep = resolveDependency(depArtifact);
+            if (!depArtifact.isResolved()) {
+                ArtifactResolutionRequest arr = new ArtifactResolutionRequest();
+                arr.setLocalRepository(localRepository);
+                arr.setRemoteRepositories(remoteRepositories);
+                arr.setArtifact(depArtifact);
+                ArtifactResolutionResult result = artifactResolver.resolve(arr);
+                if (!result.isSuccess()) {
+                    throw new ProjectBuildingException(project.getId(),
+                            "Unable to resolve " + depArtifact + ": " + result.getExceptions(), (Throwable) null);
                 }
             }
+            dep.setArtifact(depArtifact);
+            dependencyGavMap.put(toGav(dep), dep);
+            List<Pair<String, String>> licenseUrls = new ArrayList<>();
+            for (Object license : dep.getLicenses()) {
+                final License license1 = (License) license;
+                String url = license1.getUrl() != null ? license1.getUrl()
+                        : (license1.getName() != null ? license1.getName() : "LICENSE_EMPTY_NAME_URL");
+                licenseUrls.add(new ImmutablePair<>(url, license1.getName()));
+            }
+            dependencyLicenseMap.put(dep, licenseUrls);
         }
     }
 
@@ -430,7 +462,7 @@
                     .get(SupplementalModelHelper.generateSupplementMapKey(depObj.getGroupId(), depObj.getArtifactId()));
             registerVerified(depProj, supplement);
             if (supplement != null) {
-                Model merged = SupplementalModelHelper.mergeModels(assembler, depProj.getModel(), supplement);
+                Model merged = SupplementalModelHelper.mergeModels(assembler, depProj.getModel(), supplement).clone();
                 Set<String> origLicenses =
                         depProj.getModel().getLicenses().stream().map(License::getUrl).collect(Collectors.toSet());
                 Set<String> newLicenses =
diff --git a/hyracks-fullstack/hyracks/hyracks-net/pom.xml b/hyracks-fullstack/hyracks/hyracks-net/pom.xml
index 5ebfb2e..74faf50 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-net/pom.xml
@@ -71,7 +71,7 @@
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
+      <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
   </dependencies>
diff --git a/hyracks-fullstack/hyracks/hyracks-net/src/test/java/org/apache/hyracks/net/tests/FullFrameChannelReadInterfaceTest.java b/hyracks-fullstack/hyracks/hyracks-net/src/test/java/org/apache/hyracks/net/tests/FullFrameChannelReadInterfaceTest.java
index 2180d1d..cf0ae08 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/src/test/java/org/apache/hyracks/net/tests/FullFrameChannelReadInterfaceTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-net/src/test/java/org/apache/hyracks/net/tests/FullFrameChannelReadInterfaceTest.java
@@ -114,7 +114,7 @@
         final ChannelControlBlock ccb = Mockito.mock(ChannelControlBlock.class);
         Mockito.when(ccb.isRemotelyClosed()).thenReturn(false);
         Mockito.doAnswer(invocation -> {
-            final Integer delta = invocation.getArgumentAt(0, Integer.class);
+            final Integer delta = invocation.getArgument(0);
             credit.addAndGet(delta);
             synchronized (credit) {
                 credit.notifyAll();
@@ -128,7 +128,7 @@
         final ISocketChannel sc = Mockito.mock(ISocketChannel.class);
         Mockito.when(sc.read(Mockito.any(ByteBuffer.class))).thenAnswer(invocation -> {
             ccb.addPendingCredits(-FRAME_SIZE);
-            final ByteBuffer buffer = invocation.getArgumentAt(0, ByteBuffer.class);
+            final ByteBuffer buffer = invocation.getArgument(0);
             while (buffer.hasRemaining()) {
                 buffer.put((byte) 0);
             }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
index 37e9959..28f8e1f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
@@ -67,25 +67,22 @@
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
+      <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.powermock</groupId>
-      <artifactId>powermock-api-mockito</artifactId>
-      <version>1.6.2</version>
+      <artifactId>powermock-api-mockito2</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.powermock</groupId>
       <artifactId>powermock-module-junit4</artifactId>
-      <version>1.6.2</version>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.powermock</groupId>
       <artifactId>powermock-core</artifactId>
-      <version>1.6.2</version>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
index 44af086..0600add 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
@@ -73,7 +73,7 @@
 import org.powermock.modules.junit4.PowerMockRunner;
 
 @RunWith(PowerMockRunner.class)
-@PowerMockIgnore("javax.management.*")
+@PowerMockIgnore({ "javax.management.*", "javax.xml.*", "com.sun.org.apache.xerces.*", "org.w3c.*", "org.xml.*" })
 @PrepareForTest({ BTreeUtils.class, FrameTupleAccessor.class, ArrayTupleBuilder.class,
         IndexSearchOperatorNodePushable.class, FrameUtils.class, FrameTupleAppender.class })
 public class FramewriterTest {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
index af7ec3d..6a67dc3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
@@ -102,7 +102,7 @@
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
+      <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java
index e48db2b..3b6669e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java
@@ -21,6 +21,7 @@
 
 import static org.apache.hyracks.api.exceptions.ErrorCode.CANNOT_DROP_IN_USE_INDEX;
 import static org.apache.hyracks.api.exceptions.ErrorCode.INDEX_DOES_NOT_EXIST;
+import static org.apache.hyracks.api.exceptions.ErrorCode.RESOURCE_DOES_NOT_EXIST;
 import static org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption.IF_EXISTS;
 import static org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption.WAIT_ON_IN_USE;
 
@@ -96,7 +97,7 @@
     }
 
     private boolean isIgnorable(HyracksDataException e) {
-        return e.matches(INDEX_DOES_NOT_EXIST) && options.contains(IF_EXISTS);
+        return (e.matches(INDEX_DOES_NOT_EXIST) || e.matches(RESOURCE_DOES_NOT_EXIST)) && options.contains(IF_EXISTS);
     }
 
     private boolean canRetry(HyracksDataException e) throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
index d0dfbf4..7ed6d34 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
@@ -69,7 +69,6 @@
     <dependency>
       <groupId>org.xerial.snappy</groupId>
       <artifactId>snappy-java</artifactId>
-      <version>1.1.7.1</version>
     </dependency>
     <dependency>
       <groupId>com.google.guava</groupId>
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/pom.xml b/hyracks-fullstack/hyracks/hyracks-test-support/pom.xml
index db7000c..1ba8b5b 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/pom.xml
@@ -118,7 +118,7 @@
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
+      <artifactId>mockito-core</artifactId>
     </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
index f34eb72..581ad38 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
@@ -90,7 +90,7 @@
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
+      <artifactId>mockito-core</artifactId>
       <scope>test</scope>
     </dependency>
   </dependencies>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/ConcurrentMergePolicyTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/ConcurrentMergePolicyTest.java
index 098aa51..af0be25 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/ConcurrentMergePolicyTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/ConcurrentMergePolicyTest.java
@@ -166,7 +166,7 @@
         Mockito.doAnswer(new Answer<Void>() {
             @Override
             public Void answer(InvocationOnMock invocation) throws Throwable {
-                List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(0, List.class);
+                List<ILSMDiskComponent> mergedComponents = invocation.getArgument(0);
                 mergedComponents.forEach(component -> {
                     mergedSizes.add(component.getComponentSize());
                 });
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
index 54fd3bb..2e7b4c6 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
@@ -196,7 +196,7 @@
         Mockito.doAnswer(new Answer<Void>() {
             @Override
             public Void answer(InvocationOnMock invocation) throws Throwable {
-                List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(0, List.class);
+                List<ILSMDiskComponent> mergedComponents = invocation.getArgument(0);
                 if (mergedSizes != null) {
                     mergedComponents.forEach(component -> {
                         mergedSizes.add(component.getComponentSize());
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java
index dc8bc11..1a40360 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hyracks.util;
 
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 
@@ -70,6 +71,12 @@
         }
 
         @Override
+        public boolean await(CountDownLatch latch) throws InterruptedException {
+            latch.await();
+            return true;
+        }
+
+        @Override
         public String toString() {
             return "<INFINITE>";
         }
@@ -143,6 +150,10 @@
         TimeUnit.NANOSECONDS.timedWait(monitor, remaining(TimeUnit.NANOSECONDS));
     }
 
+    public boolean await(CountDownLatch latch) throws InterruptedException {
+        return latch.await(remaining(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
+    }
+
     public void loopUntilExhausted(ThrowingAction action) throws Exception {
         loopUntilExhausted(action, 0, TimeUnit.NANOSECONDS);
     }
diff --git a/hyracks-fullstack/pom.xml b/hyracks-fullstack/pom.xml
index 600d57c..81970b4 100644
--- a/hyracks-fullstack/pom.xml
+++ b/hyracks-fullstack/pom.xml
@@ -49,7 +49,8 @@
     <root.dir>${basedir}</root.dir>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <file.encoding>UTF-8</file.encoding>
-    <source.jdk.version>1.8</source.jdk.version>
+    <source.jdk.version>11</source.jdk.version>
+    <target.jdk.version>11</target.jdk.version>
     <javac.xlint.value>all</javac.xlint.value>
     <jvm.extraargs />
     <sonar.jacoco.reportPath>${env.PWD}/target/jacoco-merged.exec</sonar.jacoco.reportPath>
@@ -72,6 +73,7 @@
     <hadoop.version>2.8.5</hadoop.version>
     <jacoco.version>0.7.6.201602180812</jacoco.version>
     <log4j.version>2.14.1</log4j.version>
+    <snappy.version>1.1.8.4</snappy.version>
 
     <implementation.title>Apache Hyracks and Algebricks - ${project.name}</implementation.title>
     <implementation.url>https://asterixdb.apache.org/</implementation.url>
@@ -266,11 +268,6 @@
         <version>${log4j.version}</version>
       </dependency>
       <dependency>
-        <groupId>org.mockito</groupId>
-        <artifactId>mockito-all</artifactId>
-        <version>2.0.2-beta</version>
-      </dependency>
-      <dependency>
         <groupId>javax.xml.bind</groupId>
         <artifactId>jaxb-api</artifactId>
         <version>2.3.0</version>
@@ -315,6 +312,31 @@
         <artifactId>log4j-1.2-api</artifactId>
         <version>2.13.1</version>
       </dependency>
+      <dependency>
+        <groupId>org.mockito</groupId>
+        <artifactId>mockito-core</artifactId>
+        <version>3.10.0</version>
+      </dependency>
+      <dependency>
+        <groupId>org.powermock</groupId>
+        <artifactId>powermock-api-mockito2</artifactId>
+        <version>2.0.9</version>
+      </dependency>
+      <dependency>
+        <groupId>org.powermock</groupId>
+        <artifactId>powermock-module-junit4</artifactId>
+        <version>2.0.9</version>
+      </dependency>
+      <dependency>
+        <groupId>org.powermock</groupId>
+        <artifactId>powermock-core</artifactId>
+        <version>2.0.9</version>
+      </dependency>
+      <dependency>
+        <groupId>org.xerial.snappy</groupId>
+        <artifactId>snappy-java</artifactId>
+        <version>${snappy.version}</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
   <build>
@@ -528,6 +550,20 @@
                 <requireMavenVersion>
                   <version>[3.3.9,)</version>
                 </requireMavenVersion>
+                <requireJavaVersion>
+                  <version>[11,)</version>
+                </requireJavaVersion>
+              </rules>
+            </configuration>
+          </execution>
+          <execution>
+            <id>ensure-no-duplicate-deps</id>
+            <goals>
+              <goal>enforce</goal>
+            </goals>
+            <configuration>
+              <rules>
+                <banDuplicatePomDependencyVersions/>
               </rules>
             </configuration>
           </execution>
@@ -724,13 +760,18 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-dependency-plugin</artifactId>
-          <version>3.1.1</version>
+          <version>3.1.2</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.doxia</groupId>
           <artifactId>doxia-maven-plugin</artifactId>
           <version>1.1.4</version>
         </plugin>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-enforcer-plugin</artifactId>
+          <version>3.0.0-M3</version>
+        </plugin>
       </plugins>
     </pluginManagement>
   </build>
@@ -863,40 +904,6 @@
         <failIfNoTests>false</failIfNoTests>
       </properties>
     </profile>
-    <profile>
-      <id>java8</id>
-      <activation>
-        <jdk>[1.8,)</jdk>
-      </activation>
-      <properties>
-        <target.jdk.version>8</target.jdk.version>
-      </properties>
-    </profile>
-    <profile>
-      <id>java8-bootclasspath</id>
-      <activation>
-        <property>
-          <name>java8-bootclasspath</name>
-        </property>
-      </activation>
-      <properties>
-        <target.jdk.version>8</target.jdk.version>
-      </properties>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-compiler-plugin</artifactId>
-            <configuration>
-              <compilerArgs combine.children="append">
-                <arg>-bootclasspath</arg>
-                <arg>${java8-bootclasspath}</arg>
-              </compilerArgs>
-            </configuration>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
   </profiles>
   <modules>
     <module>hyracks</module>