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>